Showing {actualFirst + 1}-{last + 1} of {allApps.size}
@@ -122,8 +123,10 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
"Spark User",
"Last Updated")
- private def rangeIndices(range: Seq[Int], condition: Int => Boolean): Seq[Node] = {
- range.filter(condition).map(nextPage => {nextPage} )
+ private def rangeIndices(range: Seq[Int], condition: Int => Boolean, showIncomplete: Boolean):
+ Seq[Node] = {
+ range.filter(condition).map(nextPage =>
+ {nextPage} )
}
private def appRow(info: ApplicationHistoryInfo): Seq[Node] = {
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 fa9bfe5426b6c..af483d560b33e 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
@@ -96,6 +96,10 @@ class HistoryServer(
}
}
}
+ // SPARK-5983 ensure TRACE is not supported
+ protected override def doTrace(req: HttpServletRequest, res: HttpServletResponse): Unit = {
+ res.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED)
+ }
}
initialize()
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
index ede0a9dbefb8d..a962dc4af2f6c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
@@ -90,9 +90,9 @@ private[spark] class ApplicationInfo(
}
}
- private val myMaxCores = desc.maxCores.getOrElse(defaultCores)
+ val requestedCores = desc.maxCores.getOrElse(defaultCores)
- def coresLeft: Int = myMaxCores - coresGranted
+ def coresLeft: Int = requestedCores - coresGranted
private var _retryCount = 0
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
index 67e6c5d66af0e..f5b946329ae9b 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
@@ -21,7 +21,7 @@ private[spark] object ApplicationState extends Enumeration {
type ApplicationState = Value
- val WAITING, RUNNING, FINISHED, FAILED, UNKNOWN = Value
+ val WAITING, RUNNING, FINISHED, FAILED, KILLED, UNKNOWN = Value
val MAX_NUM_RETRY = 10
}
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 d92d99310a583..22935c9b1d394 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
@@ -43,6 +43,7 @@ import org.apache.spark.deploy.history.HistoryServer
import org.apache.spark.deploy.master.DriverState.DriverState
import org.apache.spark.deploy.master.MasterMessages._
import org.apache.spark.deploy.master.ui.MasterWebUI
+import org.apache.spark.deploy.rest.StandaloneRestServer
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus}
import org.apache.spark.ui.SparkUI
@@ -52,12 +53,12 @@ private[spark] class Master(
host: String,
port: Int,
webUiPort: Int,
- val securityMgr: SecurityManager)
+ val securityMgr: SecurityManager,
+ val conf: SparkConf)
extends Actor with ActorLogReceive with Logging with LeaderElectable {
import context.dispatcher // to use Akka's scheduler.schedule()
- val conf = new SparkConf
val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf)
def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs
@@ -95,7 +96,7 @@ private[spark] class Master(
val webUi = new MasterWebUI(this, webUiPort)
val masterPublicAddress = {
- val envVar = System.getenv("SPARK_PUBLIC_DNS")
+ val envVar = conf.getenv("SPARK_PUBLIC_DNS")
if (envVar != null) envVar else host
}
@@ -121,6 +122,17 @@ private[spark] class Master(
throw new SparkException("spark.deploy.defaultCores must be positive")
}
+ // Alternative application submission gateway that is stable across Spark versions
+ private val restServerEnabled = conf.getBoolean("spark.master.rest.enabled", true)
+ private val restServer =
+ if (restServerEnabled) {
+ val port = conf.getInt("spark.master.rest.port", 6066)
+ Some(new StandaloneRestServer(host, port, self, masterUrl, conf))
+ } else {
+ None
+ }
+ private val restServerBoundPort = restServer.map(_.start())
+
override def preStart() {
logInfo("Starting Spark master at " + masterUrl)
logInfo(s"Running Spark version ${org.apache.spark.SPARK_VERSION}")
@@ -174,6 +186,7 @@ private[spark] class Master(
recoveryCompletionTask.cancel()
}
webUi.stop()
+ restServer.foreach(_.stop())
masterMetricsSystem.stop()
applicationMetricsSystem.stop()
persistenceEngine.close()
@@ -421,7 +434,9 @@ private[spark] class Master(
}
case RequestMasterState => {
- sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray,
+ sender ! MasterStateResponse(
+ host, port, restServerBoundPort,
+ workers.toArray, apps.toArray, completedApps.toArray,
drivers.toArray, completedDrivers.toArray, state)
}
@@ -429,8 +444,8 @@ private[spark] class Master(
timeOutDeadWorkers()
}
- case RequestWebUIPort => {
- sender ! WebUIPortResponse(webUi.boundPort)
+ case BoundPortsRequest => {
+ sender ! BoundPortsResponse(port, webUi.boundPort, restServerBoundPort)
}
}
@@ -656,7 +671,7 @@ private[spark] class Master(
def registerApplication(app: ApplicationInfo): Unit = {
val appAddress = app.driver.path.address
- if (addressToWorker.contains(appAddress)) {
+ if (addressToApp.contains(appAddress)) {
logInfo("Attempted to re-register application at same address: " + appAddress)
return
}
@@ -721,32 +736,37 @@ private[spark] class Master(
val appName = app.desc.name
val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found"
try {
- val eventLogFile = app.desc.eventLogDir
- .map { dir => EventLoggingListener.getLogPath(dir, app.id) }
+ val eventLogDir = app.desc.eventLogDir
.getOrElse {
// Event logging is not enabled for this application
app.desc.appUiUrl = notFoundBasePath
return false
}
-
- val fs = Utils.getHadoopFileSystem(eventLogFile, hadoopConf)
-
- if (fs.exists(new Path(eventLogFile + EventLoggingListener.IN_PROGRESS))) {
+
+ val eventLogFilePrefix = EventLoggingListener.getLogPath(
+ eventLogDir, app.id, app.desc.eventLogCodec)
+ val fs = Utils.getHadoopFileSystem(eventLogDir, hadoopConf)
+ val inProgressExists = fs.exists(new Path(eventLogFilePrefix +
+ EventLoggingListener.IN_PROGRESS))
+
+ if (inProgressExists) {
// Event logging is enabled for this application, but the application is still in progress
- val title = s"Application history not found (${app.id})"
- var msg = s"Application $appName is still in progress."
- logWarning(msg)
- msg = URLEncoder.encode(msg, "UTF-8")
- app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title"
- return false
+ logWarning(s"Application $appName is still in progress, it may be terminated abnormally.")
}
-
- val (logInput, sparkVersion) = EventLoggingListener.openEventLog(new Path(eventLogFile), fs)
+
+ val (eventLogFile, status) = if (inProgressExists) {
+ (eventLogFilePrefix + EventLoggingListener.IN_PROGRESS, " (in progress)")
+ } else {
+ (eventLogFilePrefix, " (completed)")
+ }
+
+ val logInput = EventLoggingListener.openEventLog(new Path(eventLogFile), fs)
val replayBus = new ReplayListenerBus()
val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf),
- appName + " (completed)", HistoryServer.UI_PATH_PREFIX + s"/${app.id}")
+ appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}")
+ val maybeTruncated = eventLogFile.endsWith(EventLoggingListener.IN_PROGRESS)
try {
- replayBus.replay(logInput, sparkVersion)
+ replayBus.replay(logInput, eventLogFile, maybeTruncated)
} finally {
logInput.close()
}
@@ -759,7 +779,7 @@ private[spark] class Master(
case fnf: FileNotFoundException =>
// Event logging is enabled for this application, but no event logs are found
val title = s"Application history not found (${app.id})"
- var msg = s"No event logs found for application $appName in ${app.desc.eventLogDir}."
+ var msg = s"No event logs found for application $appName in ${app.desc.eventLogDir.get}."
logWarning(msg)
msg += " Did you specify the correct logging directory?"
msg = URLEncoder.encode(msg, "UTF-8")
@@ -851,7 +871,7 @@ private[spark] object Master extends Logging {
SignalLogger.register(log)
val conf = new SparkConf
val args = new MasterArguments(argStrings, conf)
- val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort, conf)
+ val (actorSystem, _, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort, conf)
actorSystem.awaitTermination()
}
@@ -860,9 +880,9 @@ private[spark] object Master extends Logging {
*
* @throws SparkException if the url is invalid
*/
- def toAkkaUrl(sparkUrl: String): String = {
+ def toAkkaUrl(sparkUrl: String, protocol: String): String = {
val (host, port) = Utils.extractHostPortFromSparkUrl(sparkUrl)
- "akka.tcp://%s@%s:%s/user/%s".format(systemName, host, port, actorName)
+ AkkaUtils.address(protocol, systemName, host, port, actorName)
}
/**
@@ -870,24 +890,31 @@ private[spark] object Master extends Logging {
*
* @throws SparkException if the url is invalid
*/
- def toAkkaAddress(sparkUrl: String): Address = {
+ def toAkkaAddress(sparkUrl: String, protocol: String): Address = {
val (host, port) = Utils.extractHostPortFromSparkUrl(sparkUrl)
- Address("akka.tcp", systemName, host, port)
+ Address(protocol, systemName, host, port)
}
+ /**
+ * Start the Master and return a four tuple of:
+ * (1) The Master actor system
+ * (2) The bound port
+ * (3) The web UI bound port
+ * (4) The REST server bound port, if any
+ */
def startSystemAndActor(
host: String,
port: Int,
webUiPort: Int,
- conf: SparkConf): (ActorSystem, Int, Int) = {
+ conf: SparkConf): (ActorSystem, Int, Int, Option[Int]) = {
val securityMgr = new SecurityManager(conf)
val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf,
securityManager = securityMgr)
- val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort,
- securityMgr), actorName)
+ val actor = actorSystem.actorOf(
+ Props(classOf[Master], host, boundPort, webUiPort, securityMgr, conf), actorName)
val timeout = AkkaUtils.askTimeout(conf)
- val respFuture = actor.ask(RequestWebUIPort)(timeout)
- val resp = Await.result(respFuture, timeout).asInstanceOf[WebUIPortResponse]
- (actorSystem, boundPort, resp.webUIBoundPort)
+ val portsRequest = actor.ask(BoundPortsRequest)(timeout)
+ val portsResponse = Await.result(portsRequest, timeout).asInstanceOf[BoundPortsResponse]
+ (actorSystem, boundPort, portsResponse.webUIPort, portsResponse.restPort)
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala
index db72d8ae9bdaf..15c6296888f70 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala
@@ -36,7 +36,7 @@ private[master] object MasterMessages {
case object CompleteRecovery
- case object RequestWebUIPort
+ case object BoundPortsRequest
- case class WebUIPortResponse(webUIBoundPort: Int)
+ case class BoundPortsResponse(actorPort: Int, webUIPort: Int, restPort: Option[Int])
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
index 3aae2b95d7396..76fc40e17d9a8 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
@@ -24,6 +24,7 @@ import scala.xml.Node
import akka.pattern.ask
import org.json4s.JValue
+import org.json4s.JsonAST.JNothing
import org.apache.spark.deploy.{ExecutorState, JsonProtocol}
import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
@@ -44,7 +45,11 @@ private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app
val app = state.activeApps.find(_.id == appId).getOrElse({
state.completedApps.find(_.id == appId).getOrElse(null)
})
- JsonProtocol.writeApplicationInfo(app)
+ if (app == null) {
+ JNothing
+ } else {
+ JsonProtocol.writeApplicationInfo(app)
+ }
}
/** Executor details for a particular application */
@@ -55,6 +60,10 @@ private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app
val app = state.activeApps.find(_.id == appId).getOrElse({
state.completedApps.find(_.id == appId).getOrElse(null)
})
+ if (app == null) {
+ val msg = No running application with ID {appId}
+ return UIUtils.basicSparkPage(msg, "Not Found")
+ }
val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs")
val allExecutors = (app.executors.values ++ app.removedExecutors).toSet.toSeq
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
index 7ca3b08a28728..c086cadca2c7d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
@@ -26,8 +26,8 @@ import akka.pattern.ask
import org.json4s.JValue
import org.apache.spark.deploy.JsonProtocol
-import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
-import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo}
+import org.apache.spark.deploy.DeployMessages.{RequestKillDriver, MasterStateResponse, RequestMasterState}
+import org.apache.spark.deploy.master._
import org.apache.spark.ui.{WebUIPage, UIUtils}
import org.apache.spark.util.Utils
@@ -41,24 +41,53 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
JsonProtocol.writeMasterState(state)
}
+ def handleAppKillRequest(request: HttpServletRequest): Unit = {
+ handleKillRequest(request, id => {
+ parent.master.idToApp.get(id).foreach { app =>
+ parent.master.removeApplication(app, ApplicationState.KILLED)
+ }
+ })
+ }
+
+ def handleDriverKillRequest(request: HttpServletRequest): Unit = {
+ handleKillRequest(request, id => { master ! RequestKillDriver(id) })
+ }
+
+ private def handleKillRequest(request: HttpServletRequest, action: String => Unit): Unit = {
+ if (parent.killEnabled &&
+ parent.master.securityMgr.checkModifyPermissions(request.getRemoteUser)) {
+ val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean
+ val id = Option(request.getParameter("id"))
+ if (id.isDefined && killFlag) {
+ action(id.get)
+ }
+
+ Thread.sleep(100)
+ }
+ }
+
/** Index view listing applications and executors */
def render(request: HttpServletRequest): Seq[Node] = {
val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
val state = Await.result(stateFuture, timeout)
- val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory")
+ val workerHeaders = Seq("Worker Id", "Address", "State", "Cores", "Memory")
val workers = state.workers.sortBy(_.id)
val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers)
- val appHeaders = Seq("ID", "Name", "Cores", "Memory per Node", "Submitted Time", "User",
- "State", "Duration")
+ val activeAppHeaders = Seq("Application ID", "Name", "Cores in Use",
+ "Cores Requested", "Memory per Node", "Submitted Time", "User", "State", "Duration")
val activeApps = state.activeApps.sortBy(_.startTime).reverse
- val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps)
+ val activeAppsTable = UIUtils.listingTable(activeAppHeaders, activeAppRow, activeApps)
+
+ val completedAppHeaders = Seq("Application ID", "Name", "Cores Requested", "Memory per Node",
+ "Submitted Time", "User", "State", "Duration")
val completedApps = state.completedApps.sortBy(_.endTime).reverse
- val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps)
+ val completedAppsTable = UIUtils.listingTable(completedAppHeaders, completeAppRow,
+ completedApps)
- val driverHeaders = Seq("ID", "Submitted Time", "Worker", "State", "Cores", "Memory",
- "Main Class")
+ val driverHeaders = Seq("Submission ID", "Submitted Time", "Worker", "State", "Cores",
+ "Memory", "Main Class")
val activeDrivers = state.activeDrivers.sortBy(_.startTime).reverse
val activeDriversTable = UIUtils.listingTable(driverHeaders, driverRow, activeDrivers)
val completedDrivers = state.completedDrivers.sortBy(_.startTime).reverse
@@ -73,6 +102,14 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
- URL: {state.uri}
+ {
+ state.restUri.map { uri =>
+ -
+ REST URL: {uri}
+ (cluster mode)
+
+ }.getOrElse { Seq.empty }
+ }
- Workers: {state.workers.size}
- Cores: {state.workers.map(_.cores).sum} Total,
{state.workers.map(_.coresUsed).sum} Used
@@ -154,16 +191,34 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
}
- private def appRow(app: ApplicationInfo): Seq[Node] = {
+ private def appRow(app: ApplicationInfo, active: Boolean): Seq[Node] = {
+ val killLink = if (parent.killEnabled &&
+ (app.state == ApplicationState.RUNNING || app.state == ApplicationState.WAITING)) {
+ val killLinkUri = s"app/kill?id=${app.id}&terminate=true"
+ val confirm = "return window.confirm(" +
+ s"'Are you sure you want to kill application ${app.id} ?');"
+
+ (kill)
+
+ }
+
{app.id}
+ {killLink}
{app.desc.name}
+ {
+ if (active) {
+
+ {app.coresGranted}
+
+ }
+ }
- {app.coresGranted}
+ {if (app.requestedCores == Int.MaxValue) "*" else app.requestedCores}
{Utils.megabytesToString(app.desc.memoryPerSlave)}
@@ -175,9 +230,28 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
}
+ private def activeAppRow(app: ApplicationInfo): Seq[Node] = {
+ appRow(app, active = true)
+ }
+
+ private def completeAppRow(app: ApplicationInfo): Seq[Node] = {
+ appRow(app, active = false)
+ }
+
private def driverRow(driver: DriverInfo): Seq[Node] = {
+ val killLink = if (parent.killEnabled &&
+ (driver.state == DriverState.RUNNING ||
+ driver.state == DriverState.SUBMITTED ||
+ driver.state == DriverState.RELAUNCHING)) {
+ val killLinkUri = s"driver/kill?id=${driver.id}&terminate=true"
+ val confirm = "return window.confirm(" +
+ s"'Are you sure you want to kill driver ${driver.id} ?');"
+
+ (kill)
+
+ }
- {driver.id}
+ {driver.id} {killLink}
{driver.submitDate}
{driver.worker.map(w => {w.id.toString}).getOrElse("None")}
@@ -188,7 +262,7 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
{Utils.megabytesToString(driver.desc.mem.toLong)}
- {driver.desc.command.arguments(1)}
+ {driver.desc.command.arguments(2)}
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
index 73400c5affb5d..170f90a00ad2a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
@@ -32,15 +32,21 @@ class MasterWebUI(val master: Master, requestedPort: Int)
val masterActorRef = master.self
val timeout = AkkaUtils.askTimeout(master.conf)
+ val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true)
initialize()
/** Initialize all components of the server. */
def initialize() {
+ val masterPage = new MasterPage(this)
attachPage(new ApplicationPage(this))
attachPage(new HistoryNotFoundPage(this))
- attachPage(new MasterPage(this))
+ attachPage(masterPage)
attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static"))
+ attachHandler(
+ createRedirectHandler("/app/kill", "/", masterPage.handleAppKillRequest))
+ attachHandler(
+ createRedirectHandler("/driver/kill", "/", masterPage.handleDriverKillRequest))
}
/** Attach a reconstructed UI to this Master UI. Only valid after bind(). */
diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala
new file mode 100644
index 0000000000000..c4be1f19e8e9f
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala
@@ -0,0 +1,331 @@
+/*
+ * 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.rest
+
+import java.io.{DataOutputStream, FileNotFoundException}
+import java.net.{HttpURLConnection, SocketException, URL}
+import javax.servlet.http.HttpServletResponse
+
+import scala.io.Source
+
+import com.fasterxml.jackson.core.JsonProcessingException
+import com.google.common.base.Charsets
+
+import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion}
+
+/**
+ * A client that submits applications to the standalone Master using a REST protocol.
+ * This client is intended to communicate with the [[StandaloneRestServer]] and is
+ * currently used for cluster mode only.
+ *
+ * In protocol version v1, the REST URL takes the form http://[host:port]/v1/submissions/[action],
+ * where [action] can be one of create, kill, or status. Each type of request is represented in
+ * an HTTP message sent to the following prefixes:
+ * (1) submit - POST to /submissions/create
+ * (2) kill - POST /submissions/kill/[submissionId]
+ * (3) status - GET /submissions/status/[submissionId]
+ *
+ * In the case of (1), parameters are posted in the HTTP body in the form of JSON fields.
+ * Otherwise, the URL fully specifies the intended action of the client.
+ *
+ * Since the protocol is expected to be stable across Spark versions, existing fields cannot be
+ * added or removed, though new optional fields can be added. In the rare event that forward or
+ * backward compatibility is broken, Spark must introduce a new protocol version (e.g. v2).
+ *
+ * The client and the server must communicate using the same version of the protocol. If there
+ * is a mismatch, the server will respond with the highest protocol version it supports. A future
+ * implementation of this client can use that information to retry using the version specified
+ * by the server.
+ */
+private[spark] class StandaloneRestClient extends Logging {
+ import StandaloneRestClient._
+
+ /**
+ * Submit an application specified by the parameters in the provided request.
+ *
+ * If the submission was successful, poll the status of the submission and report
+ * it to the user. Otherwise, report the error message provided by the server.
+ */
+ def createSubmission(
+ master: String,
+ request: CreateSubmissionRequest): SubmitRestProtocolResponse = {
+ logInfo(s"Submitting a request to launch an application in $master.")
+ validateMaster(master)
+ val url = getSubmitUrl(master)
+ val response = postJson(url, request.toJson)
+ response match {
+ case s: CreateSubmissionResponse =>
+ reportSubmissionStatus(master, s)
+ handleRestResponse(s)
+ case unexpected =>
+ handleUnexpectedRestResponse(unexpected)
+ }
+ response
+ }
+
+ /** Request that the server kill the specified submission. */
+ def killSubmission(master: String, submissionId: String): SubmitRestProtocolResponse = {
+ logInfo(s"Submitting a request to kill submission $submissionId in $master.")
+ validateMaster(master)
+ val response = post(getKillUrl(master, submissionId))
+ response match {
+ case k: KillSubmissionResponse => handleRestResponse(k)
+ case unexpected => handleUnexpectedRestResponse(unexpected)
+ }
+ response
+ }
+
+ /** Request the status of a submission from the server. */
+ def requestSubmissionStatus(
+ master: String,
+ submissionId: String,
+ quiet: Boolean = false): SubmitRestProtocolResponse = {
+ logInfo(s"Submitting a request for the status of submission $submissionId in $master.")
+ validateMaster(master)
+ val response = get(getStatusUrl(master, submissionId))
+ response match {
+ case s: SubmissionStatusResponse => if (!quiet) { handleRestResponse(s) }
+ case unexpected => handleUnexpectedRestResponse(unexpected)
+ }
+ response
+ }
+
+ /** Construct a message that captures the specified parameters for submitting an application. */
+ def constructSubmitRequest(
+ appResource: String,
+ mainClass: String,
+ appArgs: Array[String],
+ sparkProperties: Map[String, String],
+ environmentVariables: Map[String, String]): CreateSubmissionRequest = {
+ val message = new CreateSubmissionRequest
+ message.clientSparkVersion = sparkVersion
+ message.appResource = appResource
+ message.mainClass = mainClass
+ message.appArgs = appArgs
+ message.sparkProperties = sparkProperties
+ message.environmentVariables = environmentVariables
+ message.validate()
+ message
+ }
+
+ /** Send a GET request to the specified URL. */
+ private def get(url: URL): SubmitRestProtocolResponse = {
+ logDebug(s"Sending GET request to server at $url.")
+ val conn = url.openConnection().asInstanceOf[HttpURLConnection]
+ conn.setRequestMethod("GET")
+ readResponse(conn)
+ }
+
+ /** Send a POST request to the specified URL. */
+ private def post(url: URL): SubmitRestProtocolResponse = {
+ logDebug(s"Sending POST request to server at $url.")
+ val conn = url.openConnection().asInstanceOf[HttpURLConnection]
+ conn.setRequestMethod("POST")
+ readResponse(conn)
+ }
+
+ /** Send a POST request with the given JSON as the body to the specified URL. */
+ private def postJson(url: URL, json: String): SubmitRestProtocolResponse = {
+ logDebug(s"Sending POST request to server at $url:\n$json")
+ val conn = url.openConnection().asInstanceOf[HttpURLConnection]
+ conn.setRequestMethod("POST")
+ conn.setRequestProperty("Content-Type", "application/json")
+ conn.setRequestProperty("charset", "utf-8")
+ conn.setDoOutput(true)
+ val out = new DataOutputStream(conn.getOutputStream)
+ out.write(json.getBytes(Charsets.UTF_8))
+ out.close()
+ readResponse(conn)
+ }
+
+ /**
+ * Read the response from the server and return it as a validated [[SubmitRestProtocolResponse]].
+ * If the response represents an error, report the embedded message to the user.
+ * Exposed for testing.
+ */
+ private[rest] def readResponse(connection: HttpURLConnection): SubmitRestProtocolResponse = {
+ try {
+ val dataStream =
+ if (connection.getResponseCode == HttpServletResponse.SC_OK) {
+ connection.getInputStream
+ } else {
+ connection.getErrorStream
+ }
+ // If the server threw an exception while writing a response, it will not have a body
+ if (dataStream == null) {
+ throw new SubmitRestProtocolException("Server returned empty body")
+ }
+ val responseJson = Source.fromInputStream(dataStream).mkString
+ logDebug(s"Response from the server:\n$responseJson")
+ val response = SubmitRestProtocolMessage.fromJson(responseJson)
+ response.validate()
+ response match {
+ // If the response is an error, log the message
+ case error: ErrorResponse =>
+ logError(s"Server responded with error:\n${error.message}")
+ error
+ // Otherwise, simply return the response
+ case response: SubmitRestProtocolResponse => response
+ case unexpected =>
+ throw new SubmitRestProtocolException(
+ s"Message received from server was not a response:\n${unexpected.toJson}")
+ }
+ } catch {
+ case unreachable @ (_: FileNotFoundException | _: SocketException) =>
+ throw new SubmitRestConnectionException(
+ s"Unable to connect to server ${connection.getURL}", unreachable)
+ case malformed @ (_: JsonProcessingException | _: SubmitRestProtocolException) =>
+ throw new SubmitRestProtocolException(
+ "Malformed response received from server", malformed)
+ }
+ }
+
+ /** Return the REST URL for creating a new submission. */
+ private def getSubmitUrl(master: String): URL = {
+ val baseUrl = getBaseUrl(master)
+ new URL(s"$baseUrl/create")
+ }
+
+ /** Return the REST URL for killing an existing submission. */
+ private def getKillUrl(master: String, submissionId: String): URL = {
+ val baseUrl = getBaseUrl(master)
+ new URL(s"$baseUrl/kill/$submissionId")
+ }
+
+ /** Return the REST URL for requesting the status of an existing submission. */
+ private def getStatusUrl(master: String, submissionId: String): URL = {
+ val baseUrl = getBaseUrl(master)
+ new URL(s"$baseUrl/status/$submissionId")
+ }
+
+ /** Return the base URL for communicating with the server, including the protocol version. */
+ private def getBaseUrl(master: String): String = {
+ val masterUrl = master.stripPrefix("spark://").stripSuffix("/")
+ s"http://$masterUrl/$PROTOCOL_VERSION/submissions"
+ }
+
+ /** Throw an exception if this is not standalone mode. */
+ private def validateMaster(master: String): Unit = {
+ if (!master.startsWith("spark://")) {
+ throw new IllegalArgumentException("This REST client is only supported in standalone mode.")
+ }
+ }
+
+ /** Report the status of a newly created submission. */
+ private def reportSubmissionStatus(
+ master: String,
+ submitResponse: CreateSubmissionResponse): Unit = {
+ if (submitResponse.success) {
+ val submissionId = submitResponse.submissionId
+ if (submissionId != null) {
+ logInfo(s"Submission successfully created as $submissionId. Polling submission state...")
+ pollSubmissionStatus(master, submissionId)
+ } else {
+ // should never happen
+ logError("Application successfully submitted, but submission ID was not provided!")
+ }
+ } else {
+ val failMessage = Option(submitResponse.message).map { ": " + _ }.getOrElse("")
+ logError("Application submission failed" + failMessage)
+ }
+ }
+
+ /**
+ * Poll the status of the specified submission and log it.
+ * This retries up to a fixed number of times before giving up.
+ */
+ private def pollSubmissionStatus(master: String, submissionId: String): Unit = {
+ (1 to REPORT_DRIVER_STATUS_MAX_TRIES).foreach { _ =>
+ val response = requestSubmissionStatus(master, submissionId, quiet = true)
+ val statusResponse = response match {
+ case s: SubmissionStatusResponse => s
+ case _ => return // unexpected type, let upstream caller handle it
+ }
+ if (statusResponse.success) {
+ val driverState = Option(statusResponse.driverState)
+ val workerId = Option(statusResponse.workerId)
+ val workerHostPort = Option(statusResponse.workerHostPort)
+ val exception = Option(statusResponse.message)
+ // Log driver state, if present
+ driverState match {
+ case Some(state) => logInfo(s"State of driver $submissionId is now $state.")
+ case _ => logError(s"State of driver $submissionId was not found!")
+ }
+ // Log worker node, if present
+ (workerId, workerHostPort) match {
+ case (Some(id), Some(hp)) => logInfo(s"Driver is running on worker $id at $hp.")
+ case _ =>
+ }
+ // Log exception stack trace, if present
+ exception.foreach { e => logError(e) }
+ return
+ }
+ Thread.sleep(REPORT_DRIVER_STATUS_INTERVAL)
+ }
+ logError(s"Error: Master did not recognize driver $submissionId.")
+ }
+
+ /** Log the response sent by the server in the REST application submission protocol. */
+ private def handleRestResponse(response: SubmitRestProtocolResponse): Unit = {
+ logInfo(s"Server responded with ${response.messageType}:\n${response.toJson}")
+ }
+
+ /** Log an appropriate error if the response sent by the server is not of the expected type. */
+ private def handleUnexpectedRestResponse(unexpected: SubmitRestProtocolResponse): Unit = {
+ logError(s"Error: Server responded with message of unexpected type ${unexpected.messageType}.")
+ }
+}
+
+private[spark] object StandaloneRestClient {
+ val REPORT_DRIVER_STATUS_INTERVAL = 1000
+ val REPORT_DRIVER_STATUS_MAX_TRIES = 10
+ val PROTOCOL_VERSION = "v1"
+
+ /**
+ * Submit an application, assuming Spark parameters are specified through the given config.
+ * This is abstracted to its own method for testing purposes.
+ */
+ private[rest] def run(
+ appResource: String,
+ mainClass: String,
+ appArgs: Array[String],
+ conf: SparkConf,
+ env: Map[String, String] = sys.env): SubmitRestProtocolResponse = {
+ val master = conf.getOption("spark.master").getOrElse {
+ throw new IllegalArgumentException("'spark.master' must be set.")
+ }
+ val sparkProperties = conf.getAll.toMap
+ val environmentVariables = env.filter { case (k, _) => k.startsWith("SPARK_") }
+ val client = new StandaloneRestClient
+ val submitRequest = client.constructSubmitRequest(
+ appResource, mainClass, appArgs, sparkProperties, environmentVariables)
+ client.createSubmission(master, submitRequest)
+ }
+
+ def main(args: Array[String]): Unit = {
+ if (args.size < 2) {
+ sys.error("Usage: StandaloneRestClient [app resource] [main class] [app args*]")
+ sys.exit(1)
+ }
+ val appResource = args(0)
+ val mainClass = args(1)
+ val appArgs = args.slice(2, args.size)
+ val conf = new SparkConf
+ run(appResource, mainClass, appArgs, conf)
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala
new file mode 100644
index 0000000000000..f9e0478e4f874
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala
@@ -0,0 +1,438 @@
+/*
+ * 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.rest
+
+import java.io.File
+import java.net.InetSocketAddress
+import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse}
+
+import scala.io.Source
+
+import akka.actor.ActorRef
+import com.fasterxml.jackson.core.JsonProcessingException
+import org.eclipse.jetty.server.Server
+import org.eclipse.jetty.servlet.{ServletHolder, ServletContextHandler}
+import org.eclipse.jetty.util.thread.QueuedThreadPool
+import org.json4s._
+import org.json4s.jackson.JsonMethods._
+
+import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion}
+import org.apache.spark.util.{AkkaUtils, Utils}
+import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription}
+import org.apache.spark.deploy.ClientArguments._
+
+/**
+ * A server that responds to requests submitted by the [[StandaloneRestClient]].
+ * This is intended to be embedded in the standalone Master and used in cluster mode only.
+ *
+ * This server responds with different HTTP codes depending on the situation:
+ * 200 OK - Request was processed successfully
+ * 400 BAD REQUEST - Request was malformed, not successfully validated, or of unexpected type
+ * 468 UNKNOWN PROTOCOL VERSION - Request specified a protocol this server does not understand
+ * 500 INTERNAL SERVER ERROR - Server throws an exception internally while processing the request
+ *
+ * The server always includes a JSON representation of the relevant [[SubmitRestProtocolResponse]]
+ * in the HTTP body. If an error occurs, however, the server will include an [[ErrorResponse]]
+ * instead of the one expected by the client. If the construction of this error response itself
+ * fails, the response will consist of an empty body with a response code that indicates internal
+ * server error.
+ *
+ * @param host the address this server should bind to
+ * @param requestedPort the port this server will attempt to bind to
+ * @param masterActor reference to the Master actor to which requests can be sent
+ * @param masterUrl the URL of the Master new drivers will attempt to connect to
+ * @param masterConf the conf used by the Master
+ */
+private[spark] class StandaloneRestServer(
+ host: String,
+ requestedPort: Int,
+ masterActor: ActorRef,
+ masterUrl: String,
+ masterConf: SparkConf)
+ extends Logging {
+
+ import StandaloneRestServer._
+
+ private var _server: Option[Server] = None
+
+ // A mapping from URL prefixes to servlets that serve them. Exposed for testing.
+ protected val baseContext = s"/$PROTOCOL_VERSION/submissions"
+ protected val contextToServlet = Map[String, StandaloneRestServlet](
+ s"$baseContext/create/*" -> new SubmitRequestServlet(masterActor, masterUrl, masterConf),
+ s"$baseContext/kill/*" -> new KillRequestServlet(masterActor, masterConf),
+ s"$baseContext/status/*" -> new StatusRequestServlet(masterActor, masterConf),
+ "/*" -> new ErrorServlet // default handler
+ )
+
+ /** Start the server and return the bound port. */
+ def start(): Int = {
+ val (server, boundPort) = Utils.startServiceOnPort[Server](requestedPort, doStart, masterConf)
+ _server = Some(server)
+ logInfo(s"Started REST server for submitting applications on port $boundPort")
+ boundPort
+ }
+
+ /**
+ * Map the servlets to their corresponding contexts and attach them to a server.
+ * Return a 2-tuple of the started server and the bound port.
+ */
+ private def doStart(startPort: Int): (Server, Int) = {
+ val server = new Server(new InetSocketAddress(host, startPort))
+ val threadPool = new QueuedThreadPool
+ threadPool.setDaemon(true)
+ server.setThreadPool(threadPool)
+ val mainHandler = new ServletContextHandler
+ mainHandler.setContextPath("/")
+ contextToServlet.foreach { case (prefix, servlet) =>
+ mainHandler.addServlet(new ServletHolder(servlet), prefix)
+ }
+ server.setHandler(mainHandler)
+ server.start()
+ val boundPort = server.getConnectors()(0).getLocalPort
+ (server, boundPort)
+ }
+
+ def stop(): Unit = {
+ _server.foreach(_.stop())
+ }
+}
+
+private[rest] object StandaloneRestServer {
+ val PROTOCOL_VERSION = StandaloneRestClient.PROTOCOL_VERSION
+ val SC_UNKNOWN_PROTOCOL_VERSION = 468
+}
+
+/**
+ * An abstract servlet for handling requests passed to the [[StandaloneRestServer]].
+ */
+private[rest] abstract class StandaloneRestServlet extends HttpServlet with Logging {
+
+ /**
+ * Serialize the given response message to JSON and send it through the response servlet.
+ * This validates the response before sending it to ensure it is properly constructed.
+ */
+ protected def sendResponse(
+ responseMessage: SubmitRestProtocolResponse,
+ responseServlet: HttpServletResponse): Unit = {
+ val message = validateResponse(responseMessage, responseServlet)
+ responseServlet.setContentType("application/json")
+ responseServlet.setCharacterEncoding("utf-8")
+ responseServlet.getWriter.write(message.toJson)
+ }
+
+ /**
+ * Return any fields in the client request message that the server does not know about.
+ *
+ * The mechanism for this is to reconstruct the JSON on the server side and compare the
+ * diff between this JSON and the one generated on the client side. Any fields that are
+ * only in the client JSON are treated as unexpected.
+ */
+ protected def findUnknownFields(
+ requestJson: String,
+ requestMessage: SubmitRestProtocolMessage): Array[String] = {
+ val clientSideJson = parse(requestJson)
+ val serverSideJson = parse(requestMessage.toJson)
+ val Diff(_, _, unknown) = clientSideJson.diff(serverSideJson)
+ unknown match {
+ case j: JObject => j.obj.map { case (k, _) => k }.toArray
+ case _ => Array.empty[String] // No difference
+ }
+ }
+
+ /** Return a human readable String representation of the exception. */
+ protected def formatException(e: Throwable): String = {
+ val stackTraceString = e.getStackTrace.map { "\t" + _ }.mkString("\n")
+ s"$e\n$stackTraceString"
+ }
+
+ /** Construct an error message to signal the fact that an exception has been thrown. */
+ protected def handleError(message: String): ErrorResponse = {
+ val e = new ErrorResponse
+ e.serverSparkVersion = sparkVersion
+ e.message = message
+ e
+ }
+
+ /**
+ * Parse a submission ID from the relative path, assuming it is the first part of the path.
+ * For instance, we expect the path to take the form /[submission ID]/maybe/something/else.
+ * The returned submission ID cannot be empty. If the path is unexpected, return None.
+ */
+ protected def parseSubmissionId(path: String): Option[String] = {
+ if (path == null || path.isEmpty) {
+ None
+ } else {
+ path.stripPrefix("/").split("/").headOption.filter(_.nonEmpty)
+ }
+ }
+
+ /**
+ * Validate the response to ensure that it is correctly constructed.
+ *
+ * If it is, simply return the message as is. Otherwise, return an error response instead
+ * to propagate the exception back to the client and set the appropriate error code.
+ */
+ private def validateResponse(
+ responseMessage: SubmitRestProtocolResponse,
+ responseServlet: HttpServletResponse): SubmitRestProtocolResponse = {
+ try {
+ responseMessage.validate()
+ responseMessage
+ } catch {
+ case e: Exception =>
+ responseServlet.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR)
+ handleError("Internal server error: " + formatException(e))
+ }
+ }
+}
+
+/**
+ * A servlet for handling kill requests passed to the [[StandaloneRestServer]].
+ */
+private[rest] class KillRequestServlet(masterActor: ActorRef, conf: SparkConf)
+ extends StandaloneRestServlet {
+
+ /**
+ * If a submission ID is specified in the URL, have the Master kill the corresponding
+ * driver and return an appropriate response to the client. Otherwise, return error.
+ */
+ protected override def doPost(
+ request: HttpServletRequest,
+ response: HttpServletResponse): Unit = {
+ val submissionId = parseSubmissionId(request.getPathInfo)
+ val responseMessage = submissionId.map(handleKill).getOrElse {
+ response.setStatus(HttpServletResponse.SC_BAD_REQUEST)
+ handleError("Submission ID is missing in kill request.")
+ }
+ sendResponse(responseMessage, response)
+ }
+
+ protected def handleKill(submissionId: String): KillSubmissionResponse = {
+ val askTimeout = AkkaUtils.askTimeout(conf)
+ val response = AkkaUtils.askWithReply[DeployMessages.KillDriverResponse](
+ DeployMessages.RequestKillDriver(submissionId), masterActor, askTimeout)
+ val k = new KillSubmissionResponse
+ k.serverSparkVersion = sparkVersion
+ k.message = response.message
+ k.submissionId = submissionId
+ k.success = response.success
+ k
+ }
+}
+
+/**
+ * A servlet for handling status requests passed to the [[StandaloneRestServer]].
+ */
+private[rest] class StatusRequestServlet(masterActor: ActorRef, conf: SparkConf)
+ extends StandaloneRestServlet {
+
+ /**
+ * If a submission ID is specified in the URL, request the status of the corresponding
+ * driver from the Master and include it in the response. Otherwise, return error.
+ */
+ protected override def doGet(
+ request: HttpServletRequest,
+ response: HttpServletResponse): Unit = {
+ val submissionId = parseSubmissionId(request.getPathInfo)
+ val responseMessage = submissionId.map(handleStatus).getOrElse {
+ response.setStatus(HttpServletResponse.SC_BAD_REQUEST)
+ handleError("Submission ID is missing in status request.")
+ }
+ sendResponse(responseMessage, response)
+ }
+
+ protected def handleStatus(submissionId: String): SubmissionStatusResponse = {
+ val askTimeout = AkkaUtils.askTimeout(conf)
+ val response = AkkaUtils.askWithReply[DeployMessages.DriverStatusResponse](
+ DeployMessages.RequestDriverStatus(submissionId), masterActor, askTimeout)
+ val message = response.exception.map { s"Exception from the cluster:\n" + formatException(_) }
+ val d = new SubmissionStatusResponse
+ d.serverSparkVersion = sparkVersion
+ d.submissionId = submissionId
+ d.success = response.found
+ d.driverState = response.state.map(_.toString).orNull
+ d.workerId = response.workerId.orNull
+ d.workerHostPort = response.workerHostPort.orNull
+ d.message = message.orNull
+ d
+ }
+}
+
+/**
+ * A servlet for handling submit requests passed to the [[StandaloneRestServer]].
+ */
+private[rest] class SubmitRequestServlet(
+ masterActor: ActorRef,
+ masterUrl: String,
+ conf: SparkConf)
+ extends StandaloneRestServlet {
+
+ /**
+ * Submit an application to the Master with parameters specified in the request.
+ *
+ * The request is assumed to be a [[SubmitRestProtocolRequest]] in the form of JSON.
+ * If the request is successfully processed, return an appropriate response to the
+ * client indicating so. Otherwise, return error instead.
+ */
+ protected override def doPost(
+ requestServlet: HttpServletRequest,
+ responseServlet: HttpServletResponse): Unit = {
+ val responseMessage =
+ try {
+ val requestMessageJson = Source.fromInputStream(requestServlet.getInputStream).mkString
+ val requestMessage = SubmitRestProtocolMessage.fromJson(requestMessageJson)
+ // The response should have already been validated on the client.
+ // In case this is not true, validate it ourselves to avoid potential NPEs.
+ requestMessage.validate()
+ handleSubmit(requestMessageJson, requestMessage, responseServlet)
+ } catch {
+ // The client failed to provide a valid JSON, so this is not our fault
+ case e @ (_: JsonProcessingException | _: SubmitRestProtocolException) =>
+ responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST)
+ handleError("Malformed request: " + formatException(e))
+ }
+ sendResponse(responseMessage, responseServlet)
+ }
+
+ /**
+ * Handle the submit request and construct an appropriate response to return to the client.
+ *
+ * This assumes that the request message is already successfully validated.
+ * If the request message is not of the expected type, return error to the client.
+ */
+ private def handleSubmit(
+ requestMessageJson: String,
+ requestMessage: SubmitRestProtocolMessage,
+ responseServlet: HttpServletResponse): SubmitRestProtocolResponse = {
+ requestMessage match {
+ case submitRequest: CreateSubmissionRequest =>
+ val askTimeout = AkkaUtils.askTimeout(conf)
+ val driverDescription = buildDriverDescription(submitRequest)
+ val response = AkkaUtils.askWithReply[DeployMessages.SubmitDriverResponse](
+ DeployMessages.RequestSubmitDriver(driverDescription), masterActor, askTimeout)
+ val submitResponse = new CreateSubmissionResponse
+ submitResponse.serverSparkVersion = sparkVersion
+ submitResponse.message = response.message
+ submitResponse.success = response.success
+ submitResponse.submissionId = response.driverId.orNull
+ val unknownFields = findUnknownFields(requestMessageJson, requestMessage)
+ if (unknownFields.nonEmpty) {
+ // If there are fields that the server does not know about, warn the client
+ submitResponse.unknownFields = unknownFields
+ }
+ submitResponse
+ case unexpected =>
+ responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST)
+ handleError(s"Received message of unexpected type ${unexpected.messageType}.")
+ }
+ }
+
+ /**
+ * Build a driver description from the fields specified in the submit request.
+ *
+ * This involves constructing a command that takes into account memory, java options,
+ * classpath and other settings to launch the driver. This does not currently consider
+ * fields used by python applications since python is not supported in standalone
+ * cluster mode yet.
+ */
+ private def buildDriverDescription(request: CreateSubmissionRequest): DriverDescription = {
+ // Required fields, including the main class because python is not yet supported
+ val appResource = Option(request.appResource).getOrElse {
+ throw new SubmitRestMissingFieldException("Application jar is missing.")
+ }
+ val mainClass = Option(request.mainClass).getOrElse {
+ throw new SubmitRestMissingFieldException("Main class is missing.")
+ }
+
+ // Optional fields
+ val sparkProperties = request.sparkProperties
+ val driverMemory = sparkProperties.get("spark.driver.memory")
+ val driverCores = sparkProperties.get("spark.driver.cores")
+ val driverExtraJavaOptions = sparkProperties.get("spark.driver.extraJavaOptions")
+ val driverExtraClassPath = sparkProperties.get("spark.driver.extraClassPath")
+ val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath")
+ val superviseDriver = sparkProperties.get("spark.driver.supervise")
+ val appArgs = request.appArgs
+ val environmentVariables = request.environmentVariables
+
+ // Construct driver description
+ val conf = new SparkConf(false)
+ .setAll(sparkProperties)
+ .set("spark.master", masterUrl)
+ val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator))
+ val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator))
+ val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty)
+ val sparkJavaOpts = Utils.sparkJavaOpts(conf)
+ val javaOpts = sparkJavaOpts ++ extraJavaOpts
+ val command = new Command(
+ "org.apache.spark.deploy.worker.DriverWrapper",
+ Seq("{{WORKER_URL}}", "{{USER_JAR}}", mainClass) ++ appArgs, // args to the DriverWrapper
+ environmentVariables, extraClassPath, extraLibraryPath, javaOpts)
+ val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY)
+ val actualDriverCores = driverCores.map(_.toInt).getOrElse(DEFAULT_CORES)
+ val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE)
+ new DriverDescription(
+ appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, command)
+ }
+}
+
+/**
+ * A default servlet that handles error cases that are not captured by other servlets.
+ */
+private class ErrorServlet extends StandaloneRestServlet {
+ private val serverVersion = StandaloneRestServer.PROTOCOL_VERSION
+
+ /** Service a faulty request by returning an appropriate error message to the client. */
+ protected override def service(
+ request: HttpServletRequest,
+ response: HttpServletResponse): Unit = {
+ val path = request.getPathInfo
+ val parts = path.stripPrefix("/").split("/").filter(_.nonEmpty).toList
+ var versionMismatch = false
+ var msg =
+ parts match {
+ case Nil =>
+ // http://host:port/
+ "Missing protocol version."
+ case `serverVersion` :: Nil =>
+ // http://host:port/correct-version
+ "Missing the /submissions prefix."
+ case `serverVersion` :: "submissions" :: tail =>
+ // http://host:port/correct-version/submissions/*
+ "Missing an action: please specify one of /create, /kill, or /status."
+ case unknownVersion :: tail =>
+ // http://host:port/unknown-version/*
+ versionMismatch = true
+ s"Unknown protocol version '$unknownVersion'."
+ case _ =>
+ // never reached
+ s"Malformed path $path."
+ }
+ msg += s" Please submit requests through http://[host]:[port]/$serverVersion/submissions/..."
+ val error = handleError(msg)
+ // If there is a version mismatch, include the highest protocol version that
+ // this server supports in case the client wants to retry with our version
+ if (versionMismatch) {
+ error.highestProtocolVersion = serverVersion
+ response.setStatus(StandaloneRestServer.SC_UNKNOWN_PROTOCOL_VERSION)
+ } else {
+ response.setStatus(HttpServletResponse.SC_BAD_REQUEST)
+ }
+ sendResponse(error, response)
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala
new file mode 100644
index 0000000000000..d7a0bdbe10778
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala
@@ -0,0 +1,36 @@
+/*
+ * 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.rest
+
+/**
+ * An exception thrown in the REST application submission protocol.
+ */
+private[spark] class SubmitRestProtocolException(message: String, cause: Throwable = null)
+ extends Exception(message, cause)
+
+/**
+ * An exception thrown if a field is missing from a [[SubmitRestProtocolMessage]].
+ */
+private[spark] class SubmitRestMissingFieldException(message: String)
+ extends SubmitRestProtocolException(message)
+
+/**
+ * An exception thrown if the REST client cannot reach the REST server.
+ */
+private[spark] class SubmitRestConnectionException(message: String, cause: Throwable)
+ extends SubmitRestProtocolException(message, cause)
diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala
new file mode 100644
index 0000000000000..8f36635674a28
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala
@@ -0,0 +1,146 @@
+/*
+ * 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.rest
+
+import com.fasterxml.jackson.annotation._
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility
+import com.fasterxml.jackson.annotation.JsonInclude.Include
+import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper, SerializationFeature}
+import com.fasterxml.jackson.module.scala.DefaultScalaModule
+import org.json4s.JsonAST._
+import org.json4s.jackson.JsonMethods._
+
+import org.apache.spark.util.Utils
+
+/**
+ * An abstract message exchanged in the REST application submission protocol.
+ *
+ * This message is intended to be serialized to and deserialized from JSON in the exchange.
+ * Each message can either be a request or a response and consists of three common fields:
+ * (1) the action, which fully specifies the type of the message
+ * (2) the Spark version of the client / server
+ * (3) an optional message
+ */
+@JsonInclude(Include.NON_NULL)
+@JsonAutoDetect(getterVisibility = Visibility.ANY, setterVisibility = Visibility.ANY)
+@JsonPropertyOrder(alphabetic = true)
+private[spark] abstract class SubmitRestProtocolMessage {
+ @JsonIgnore
+ val messageType = Utils.getFormattedClassName(this)
+
+ val action: String = messageType
+ var message: String = null
+
+ // For JSON deserialization
+ private def setAction(a: String): Unit = { }
+
+ /**
+ * Serialize the message to JSON.
+ * This also ensures that the message is valid and its fields are in the expected format.
+ */
+ def toJson: String = {
+ validate()
+ SubmitRestProtocolMessage.mapper.writeValueAsString(this)
+ }
+
+ /**
+ * Assert the validity of the message.
+ * If the validation fails, throw a [[SubmitRestProtocolException]].
+ */
+ final def validate(): Unit = {
+ try {
+ doValidate()
+ } catch {
+ case e: Exception =>
+ throw new SubmitRestProtocolException(s"Validation of message $messageType failed!", e)
+ }
+ }
+
+ /** Assert the validity of the message */
+ protected def doValidate(): Unit = {
+ if (action == null) {
+ throw new SubmitRestMissingFieldException(s"The action field is missing in $messageType")
+ }
+ }
+
+ /** Assert that the specified field is set in this message. */
+ protected def assertFieldIsSet[T](value: T, name: String): Unit = {
+ if (value == null) {
+ throw new SubmitRestMissingFieldException(s"'$name' is missing in message $messageType.")
+ }
+ }
+
+ /**
+ * Assert a condition when validating this message.
+ * If the assertion fails, throw a [[SubmitRestProtocolException]].
+ */
+ protected def assert(condition: Boolean, failMessage: String): Unit = {
+ if (!condition) { throw new SubmitRestProtocolException(failMessage) }
+ }
+}
+
+/**
+ * Helper methods to process serialized [[SubmitRestProtocolMessage]]s.
+ */
+private[spark] object SubmitRestProtocolMessage {
+ private val packagePrefix = this.getClass.getPackage.getName
+ private val mapper = new ObjectMapper()
+ .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false)
+ .enable(SerializationFeature.INDENT_OUTPUT)
+ .registerModule(DefaultScalaModule)
+
+ /**
+ * Parse the value of the action field from the given JSON.
+ * If the action field is not found, throw a [[SubmitRestMissingFieldException]].
+ */
+ def parseAction(json: String): String = {
+ val value: Option[String] = parse(json) match {
+ case JObject(fields) =>
+ fields.collectFirst { case ("action", v) => v }.collect { case JString(s) => s }
+ case _ => None
+ }
+ value.getOrElse {
+ throw new SubmitRestMissingFieldException(s"Action field not found in JSON:\n$json")
+ }
+ }
+
+ /**
+ * Construct a [[SubmitRestProtocolMessage]] from its JSON representation.
+ *
+ * This method first parses the action from the JSON and uses it to infer the message type.
+ * Note that the action must represent one of the [[SubmitRestProtocolMessage]]s defined in
+ * this package. Otherwise, a [[ClassNotFoundException]] will be thrown.
+ */
+ def fromJson(json: String): SubmitRestProtocolMessage = {
+ val className = parseAction(json)
+ val clazz = Class.forName(packagePrefix + "." + className)
+ .asSubclass[SubmitRestProtocolMessage](classOf[SubmitRestProtocolMessage])
+ fromJson(json, clazz)
+ }
+
+ /**
+ * Construct a [[SubmitRestProtocolMessage]] from its JSON representation.
+ *
+ * This method determines the type of the message from the class provided instead of
+ * inferring it from the action field. This is useful for deserializing JSON that
+ * represents custom user-defined messages.
+ */
+ def fromJson[T <: SubmitRestProtocolMessage](json: String, clazz: Class[T]): T = {
+ mapper.readValue(json, clazz)
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala
new file mode 100644
index 0000000000000..9e1fd8c40cabd
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala
@@ -0,0 +1,78 @@
+/*
+ * 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.rest
+
+import scala.util.Try
+
+import org.apache.spark.util.Utils
+
+/**
+ * An abstract request sent from the client in the REST application submission protocol.
+ */
+private[spark] abstract class SubmitRestProtocolRequest extends SubmitRestProtocolMessage {
+ var clientSparkVersion: String = null
+ protected override def doValidate(): Unit = {
+ super.doValidate()
+ assertFieldIsSet(clientSparkVersion, "clientSparkVersion")
+ }
+}
+
+/**
+ * A request to launch a new application in the REST application submission protocol.
+ */
+private[spark] class CreateSubmissionRequest extends SubmitRestProtocolRequest {
+ var appResource: String = null
+ var mainClass: String = null
+ var appArgs: Array[String] = null
+ var sparkProperties: Map[String, String] = null
+ var environmentVariables: Map[String, String] = null
+
+ protected override def doValidate(): Unit = {
+ super.doValidate()
+ assert(sparkProperties != null, "No Spark properties set!")
+ assertFieldIsSet(appResource, "appResource")
+ assertPropertyIsSet("spark.app.name")
+ assertPropertyIsBoolean("spark.driver.supervise")
+ assertPropertyIsNumeric("spark.driver.cores")
+ assertPropertyIsNumeric("spark.cores.max")
+ assertPropertyIsMemory("spark.driver.memory")
+ assertPropertyIsMemory("spark.executor.memory")
+ }
+
+ private def assertPropertyIsSet(key: String): Unit =
+ assertFieldIsSet(sparkProperties.getOrElse(key, null), key)
+
+ private def assertPropertyIsBoolean(key: String): Unit =
+ assertProperty[Boolean](key, "boolean", _.toBoolean)
+
+ private def assertPropertyIsNumeric(key: String): Unit =
+ assertProperty[Int](key, "numeric", _.toInt)
+
+ private def assertPropertyIsMemory(key: String): Unit =
+ assertProperty[Int](key, "memory", Utils.memoryStringToMb)
+
+ /** Assert that a Spark property can be converted to a certain type. */
+ private def assertProperty[T](key: String, valueType: String, convert: (String => T)): Unit = {
+ sparkProperties.get(key).foreach { value =>
+ Try(convert(value)).getOrElse {
+ throw new SubmitRestProtocolException(
+ s"Property '$key' expected $valueType value: actual was '$value'.")
+ }
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala
new file mode 100644
index 0000000000000..16dfe041d4bea
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala
@@ -0,0 +1,85 @@
+/*
+ * 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.rest
+
+import java.lang.Boolean
+
+/**
+ * An abstract response sent from the server in the REST application submission protocol.
+ */
+private[spark] abstract class SubmitRestProtocolResponse extends SubmitRestProtocolMessage {
+ var serverSparkVersion: String = null
+ var success: Boolean = null
+ var unknownFields: Array[String] = null
+ protected override def doValidate(): Unit = {
+ super.doValidate()
+ assertFieldIsSet(serverSparkVersion, "serverSparkVersion")
+ }
+}
+
+/**
+ * A response to a [[CreateSubmissionRequest]] in the REST application submission protocol.
+ */
+private[spark] class CreateSubmissionResponse extends SubmitRestProtocolResponse {
+ var submissionId: String = null
+ protected override def doValidate(): Unit = {
+ super.doValidate()
+ assertFieldIsSet(success, "success")
+ }
+}
+
+/**
+ * A response to a kill request in the REST application submission protocol.
+ */
+private[spark] class KillSubmissionResponse extends SubmitRestProtocolResponse {
+ var submissionId: String = null
+ protected override def doValidate(): Unit = {
+ super.doValidate()
+ assertFieldIsSet(submissionId, "submissionId")
+ assertFieldIsSet(success, "success")
+ }
+}
+
+/**
+ * A response to a status request in the REST application submission protocol.
+ */
+private[spark] class SubmissionStatusResponse extends SubmitRestProtocolResponse {
+ var submissionId: String = null
+ var driverState: String = null
+ var workerId: String = null
+ var workerHostPort: String = null
+
+ protected override def doValidate(): Unit = {
+ super.doValidate()
+ assertFieldIsSet(submissionId, "submissionId")
+ assertFieldIsSet(success, "success")
+ }
+}
+
+/**
+ * An error response message used in the REST application submission protocol.
+ */
+private[spark] class ErrorResponse extends SubmitRestProtocolResponse {
+ // The highest protocol version that the server knows about
+ // This is set when the client specifies an unknown version
+ var highestProtocolVersion: String = null
+ protected override def doValidate(): Unit = {
+ super.doValidate()
+ assertFieldIsSet(message, "message")
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
index 28e9662db5da9..83f78cf47306c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
@@ -20,10 +20,12 @@ package org.apache.spark.deploy.worker
import java.io.{File, FileOutputStream, InputStream, IOException}
import java.lang.System._
+import scala.collection.JavaConversions._
import scala.collection.Map
import org.apache.spark.Logging
import org.apache.spark.deploy.Command
+import org.apache.spark.launcher.WorkerCommandBuilder
import org.apache.spark.util.Utils
/**
@@ -54,12 +56,10 @@ object CommandUtils extends Logging {
}
private def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = {
- val runner = sys.env.get("JAVA_HOME").map(_ + "/bin/java").getOrElse("java")
-
// SPARK-698: do not call the run.cmd script, as process.destroy()
// fails to kill a process tree on Windows
- Seq(runner) ++ buildJavaOpts(command, memory, sparkHome) ++ Seq(command.mainClass) ++
- command.arguments
+ val cmd = new WorkerCommandBuilder(sparkHome, memory, command).buildCommand()
+ cmd.toSeq ++ Seq(command.mainClass) ++ command.arguments
}
/**
@@ -92,34 +92,6 @@ object CommandUtils extends Logging {
command.javaOpts)
}
- /**
- * Attention: this must always be aligned with the environment variables in the run scripts and
- * the way the JAVA_OPTS are assembled there.
- */
- private def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = {
- val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M")
-
- // Exists for backwards compatibility with older Spark versions
- val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString)
- .getOrElse(Nil)
- if (workerLocalOpts.length > 0) {
- logWarning("SPARK_JAVA_OPTS was set on the worker. It is deprecated in Spark 1.0.")
- logWarning("Set SPARK_LOCAL_DIRS for node-specific storage locations.")
- }
-
- // Figure out our classpath with the external compute-classpath script
- val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh"
- val classPath = Utils.executeAndGetOutput(
- Seq(sparkHome + "/bin/compute-classpath" + ext),
- extraEnvironment = command.environment)
- val userClassPath = command.classPathEntries ++ Seq(classPath)
-
- val javaVersion = System.getProperty("java.version")
- val permGenOpt = if (!javaVersion.startsWith("1.8")) Some("-XX:MaxPermSize=128m") else None
- Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++
- permGenOpt ++ workerLocalOpts ++ command.javaOpts ++ memoryOpts
- }
-
/** Spawn a thread that will redirect a given stream to a file */
def redirectStream(in: InputStream, file: File) {
val out = new FileOutputStream(file, true)
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index 28cab36c7b9e2..e16bccb24d2c4 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -20,19 +20,18 @@ package org.apache.spark.deploy.worker
import java.io._
import scala.collection.JavaConversions._
-import scala.collection.Map
import akka.actor.ActorRef
import com.google.common.base.Charsets.UTF_8
import com.google.common.io.Files
-import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileUtil, Path}
import org.apache.spark.{Logging, SparkConf}
-import org.apache.spark.deploy.{Command, DriverDescription, SparkHadoopUtil}
+import org.apache.spark.deploy.{DriverDescription, SparkHadoopUtil}
import org.apache.spark.deploy.DeployMessages.DriverStateChanged
import org.apache.spark.deploy.master.DriverState
import org.apache.spark.deploy.master.DriverState.DriverState
+import org.apache.spark.util.{Clock, SystemClock}
/**
* Manages the execution of one driver, including automatically restarting the driver on failure.
@@ -59,9 +58,7 @@ private[spark] class DriverRunner(
// Decoupled for testing
private[deploy] def setClock(_clock: Clock) = clock = _clock
private[deploy] def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper
- private var clock = new Clock {
- def currentTimeMillis(): Long = System.currentTimeMillis()
- }
+ private var clock: Clock = new SystemClock()
private var sleeper = new Sleeper {
def sleep(seconds: Int): Unit = (0 until seconds).takeWhile(f => {Thread.sleep(1000); !killed})
}
@@ -74,10 +71,15 @@ private[spark] class DriverRunner(
val driverDir = createWorkingDirectory()
val localJarFilename = downloadUserJar(driverDir)
- // Make sure user application jar is on the classpath
+ def substituteVariables(argument: String): String = argument match {
+ case "{{WORKER_URL}}" => workerUrl
+ case "{{USER_JAR}}" => localJarFilename
+ case other => other
+ }
+
// TODO: If we add ability to submit multiple jars they should also be added here
val builder = CommandUtils.buildProcessBuilder(driverDesc.command, driverDesc.mem,
- sparkHome.getAbsolutePath, substituteVariables, Seq(localJarFilename))
+ sparkHome.getAbsolutePath, substituteVariables)
launchDriver(builder, driverDir, driverDesc.supervise)
}
catch {
@@ -111,12 +113,6 @@ private[spark] class DriverRunner(
}
}
- /** Replace variables in a command argument passed to us */
- private def substituteVariables(argument: String): String = argument match {
- case "{{WORKER_URL}}" => workerUrl
- case other => other
- }
-
/**
* Creates the working directory for this driver.
* Will throw an exception if there are errors preparing the directory.
@@ -191,9 +187,9 @@ private[spark] class DriverRunner(
initialize(process.get)
}
- val processStart = clock.currentTimeMillis()
+ val processStart = clock.getTimeMillis()
val exitCode = process.get.waitFor()
- if (clock.currentTimeMillis() - processStart > successfulRunDuration * 1000) {
+ if (clock.getTimeMillis() - processStart > successfulRunDuration * 1000) {
waitSeconds = 1
}
@@ -209,10 +205,6 @@ private[spark] class DriverRunner(
}
}
-private[deploy] trait Clock {
- def currentTimeMillis(): Long
-}
-
private[deploy] trait Sleeper {
def sleep(seconds: Int)
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
index 05e242e6df702..deef6ef9043c6 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
@@ -17,32 +17,51 @@
package org.apache.spark.deploy.worker
+import java.io.File
+
import akka.actor._
import org.apache.spark.{SecurityManager, SparkConf}
-import org.apache.spark.util.{AkkaUtils, Utils}
+import org.apache.spark.util.{AkkaUtils, ChildFirstURLClassLoader, MutableURLClassLoader, Utils}
/**
* Utility object for launching driver programs such that they share fate with the Worker process.
+ * This is used in standalone cluster mode only.
*/
object DriverWrapper {
def main(args: Array[String]) {
args.toList match {
- case workerUrl :: mainClass :: extraArgs =>
+ /*
+ * IMPORTANT: Spark 1.3 provides a stable application submission gateway that is both
+ * backward and forward compatible across future Spark versions. Because this gateway
+ * uses this class to launch the driver, the ordering and semantics of the arguments
+ * here must also remain consistent across versions.
+ */
+ case workerUrl :: userJar :: mainClass :: extraArgs =>
val conf = new SparkConf()
val (actorSystem, _) = AkkaUtils.createActorSystem("Driver",
Utils.localHostName(), 0, conf, new SecurityManager(conf))
actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher")
+ val currentLoader = Thread.currentThread.getContextClassLoader
+ val userJarUrl = new File(userJar).toURI().toURL()
+ val loader =
+ if (sys.props.getOrElse("spark.driver.userClassPathFirst", "false").toBoolean) {
+ new ChildFirstURLClassLoader(Array(userJarUrl), currentLoader)
+ } else {
+ new MutableURLClassLoader(Array(userJarUrl), currentLoader)
+ }
+ Thread.currentThread.setContextClassLoader(loader)
+
// Delegate to supplied main class
- val clazz = Class.forName(args(1))
+ val clazz = Class.forName(mainClass, true, loader)
val mainMethod = clazz.getMethod("main", classOf[Array[String]])
mainMethod.invoke(null, extraArgs.toArray[String])
actorSystem.shutdown()
case _ =>
- System.err.println("Usage: DriverWrapper [options]")
+ System.err.println("Usage: DriverWrapper [options]")
System.exit(-1)
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index acbdf0d8bd7bc..023f3c6269062 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
@@ -26,7 +26,7 @@ import com.google.common.base.Charsets.UTF_8
import com.google.common.io.Files
import org.apache.spark.{SparkConf, Logging}
-import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState}
+import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged
import org.apache.spark.util.logging.FileAppender
@@ -43,6 +43,8 @@ private[spark] class ExecutorRunner(
val worker: ActorRef,
val workerId: String,
val host: String,
+ val webUiPort: Int,
+ val publicAddress: String,
val sparkHome: File,
val executorDir: File,
val workerUrl: String,
@@ -84,14 +86,13 @@ private[spark] class ExecutorRunner(
var exitCode: Option[Int] = None
if (process != null) {
logInfo("Killing process!")
- process.destroy()
- process.waitFor()
if (stdoutAppender != null) {
stdoutAppender.stop()
}
if (stderrAppender != null) {
stderrAppender.stop()
}
+ process.destroy()
exitCode = Some(process.waitFor())
}
worker ! ExecutorStateChanged(appId, execId, state, message, exitCode)
@@ -104,7 +105,11 @@ private[spark] class ExecutorRunner(
workerThread.interrupt()
workerThread = null
state = ExecutorState.KILLED
- Runtime.getRuntime.removeShutdownHook(shutdownHook)
+ try {
+ Runtime.getRuntime.removeShutdownHook(shutdownHook)
+ } catch {
+ case e: IllegalStateException => None
+ }
}
}
@@ -130,10 +135,17 @@ private[spark] class ExecutorRunner(
logInfo("Launch command: " + command.mkString("\"", "\" \"", "\""))
builder.directory(executorDir)
- builder.environment.put("SPARK_LOCAL_DIRS", appLocalDirs.mkString(","))
+ builder.environment.put("SPARK_EXECUTOR_DIRS", appLocalDirs.mkString(File.pathSeparator))
// In case we are running this from within the Spark Shell, avoid creating a "scala"
// parent process for the executor command
builder.environment.put("SPARK_LAUNCH_WITH_SCALA", "0")
+
+ // Add webUI log urls
+ val baseUrl =
+ s"http://$publicAddress:$webUiPort/logPage/?appId=$appId&executorId=$execId&logType="
+ builder.environment.put("SPARK_LOG_URL_STDERR", s"${baseUrl}stderr")
+ builder.environment.put("SPARK_LOG_URL_STDOUT", s"${baseUrl}stdout")
+
process = builder.start()
val header = "Spark Executor Command: %s\n%s\n\n".format(
command.mkString("\"", "\" \"", "\""), "=" * 40)
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 13599830123d0..f2e7418f4bf15 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
@@ -31,8 +31,8 @@ import scala.util.Random
import akka.actor._
import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
-import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException}
-import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
+import org.apache.spark.{Logging, SecurityManager, SparkConf}
+import org.apache.spark.deploy.{Command, ExecutorDescription, ExecutorState}
import org.apache.spark.deploy.DeployMessages._
import org.apache.spark.deploy.master.{DriverState, Master}
import org.apache.spark.deploy.worker.ui.WorkerWebUI
@@ -93,7 +93,12 @@ private[spark] class Worker(
var masterAddress: Address = null
var activeMasterUrl: String = ""
var activeMasterWebUiUrl : String = ""
- val akkaUrl = "akka.tcp://%s@%s:%s/user/%s".format(actorSystemName, host, port, actorName)
+ val akkaUrl = AkkaUtils.address(
+ AkkaUtils.protocol(context.system),
+ actorSystemName,
+ host,
+ port,
+ actorName)
@volatile var registered = false
@volatile var connected = false
val workerId = generateWorkerId()
@@ -116,7 +121,7 @@ private[spark] class Worker(
val shuffleService = new StandaloneWorkerShuffleService(conf, securityMgr)
val publicAddress = {
- val envVar = System.getenv("SPARK_PUBLIC_DNS")
+ val envVar = conf.getenv("SPARK_PUBLIC_DNS")
if (envVar != null) envVar else host
}
var webUi: WorkerWebUI = null
@@ -174,8 +179,9 @@ private[spark] class Worker(
// activeMasterUrl it's a valid Spark url since we receive it from master.
activeMasterUrl = url
activeMasterWebUiUrl = uiUrl
- master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl))
- masterAddress = Master.toAkkaAddress(activeMasterUrl)
+ master = context.actorSelection(
+ Master.toAkkaUrl(activeMasterUrl, AkkaUtils.protocol(context.system)))
+ masterAddress = Master.toAkkaAddress(activeMasterUrl, AkkaUtils.protocol(context.system))
connected = true
// Cancel any outstanding re-registration attempts because we found a new master
registrationRetryTimer.foreach(_.cancel())
@@ -339,18 +345,30 @@ private[spark] class Worker(
}
// Create local dirs for the executor. These are passed to the executor via the
- // SPARK_LOCAL_DIRS environment variable, and deleted by the Worker when the
+ // SPARK_EXECUTOR_DIRS environment variable, and deleted by the Worker when the
// application finishes.
val appLocalDirs = appDirectories.get(appId).getOrElse {
Utils.getOrCreateLocalRootDirs(conf).map { dir =>
- Utils.createDirectory(dir).getAbsolutePath()
+ Utils.createDirectory(dir, namePrefix = "executor").getAbsolutePath()
}.toSeq
}
appDirectories(appId) = appLocalDirs
-
- val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_,
- self, workerId, host, sparkHome, executorDir, akkaUrl, conf, appLocalDirs,
- ExecutorState.LOADING)
+ val manager = new ExecutorRunner(
+ appId,
+ execId,
+ appDesc.copy(command = Worker.maybeUpdateSSLSettings(appDesc.command, conf)),
+ cores_,
+ memory_,
+ self,
+ workerId,
+ host,
+ webUi.boundPort,
+ publicAddress,
+ sparkHome,
+ executorDir,
+ akkaUrl,
+ conf,
+ appLocalDirs, ExecutorState.LOADING)
executors(appId + "/" + execId) = manager
manager.start()
coresUsed += cores_
@@ -406,7 +424,14 @@ private[spark] class Worker(
case LaunchDriver(driverId, driverDesc) => {
logInfo(s"Asked to launch driver $driverId")
- val driver = new DriverRunner(conf, driverId, workDir, sparkHome, driverDesc, self, akkaUrl)
+ val driver = new DriverRunner(
+ conf,
+ driverId,
+ workDir,
+ sparkHome,
+ driverDesc.copy(command = Worker.maybeUpdateSSLSettings(driverDesc.command, conf)),
+ self,
+ akkaUrl)
drivers(driverId) = driver
driver.start()
@@ -514,19 +539,41 @@ private[spark] object Worker extends Logging {
memory: Int,
masterUrls: Array[String],
workDir: String,
- workerNumber: Option[Int] = None): (ActorSystem, Int) = {
+ workerNumber: Option[Int] = None,
+ conf: SparkConf = new SparkConf): (ActorSystem, Int) = {
// The LocalSparkCluster runs multiple local sparkWorkerX actor systems
- val conf = new SparkConf
val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("")
val actorName = "Worker"
val securityMgr = new SecurityManager(conf)
val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port,
conf = conf, securityManager = securityMgr)
- val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl)
+ val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl(_, AkkaUtils.protocol(actorSystem)))
actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory,
masterAkkaUrls, systemName, actorName, workDir, conf, securityMgr), name = actorName)
(actorSystem, boundPort)
}
+ private[spark] def isUseLocalNodeSSLConfig(cmd: Command): Boolean = {
+ val pattern = """\-Dspark\.ssl\.useNodeLocalConf\=(.+)""".r
+ val result = cmd.javaOpts.collectFirst {
+ case pattern(_result) => _result.toBoolean
+ }
+ result.getOrElse(false)
+ }
+
+ private[spark] def maybeUpdateSSLSettings(cmd: Command, conf: SparkConf): Command = {
+ val prefix = "spark.ssl."
+ val useNLC = "spark.ssl.useNodeLocalConf"
+ if (isUseLocalNodeSSLConfig(cmd)) {
+ val newJavaOpts = cmd.javaOpts
+ .filter(opt => !opt.startsWith(s"-D$prefix")) ++
+ conf.getAll.collect { case (key, value) if key.startsWith(prefix) => s"-D$key=$value" } :+
+ s"-D$useNLC=true"
+ cmd.copy(javaOpts = newJavaOpts)
+ } else {
+ cmd
+ }
+ }
+
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala
index 327b905032800..720f13bfa829b 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala
@@ -134,7 +134,7 @@ private[spark] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") {
def driverRow(driver: DriverRunner): Seq[Node] = {
{driver.driverId}
- {driver.driverDesc.command.arguments(1)}
+ {driver.driverDesc.command.arguments(2)}
{driver.finalState.getOrElse(DriverState.RUNNING)}
{driver.driverDesc.cores.toString}
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 823825302658c..dd19e4947db1e 100644
--- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -17,8 +17,10 @@
package org.apache.spark.executor
+import java.net.URL
import java.nio.ByteBuffer
+import scala.collection.mutable
import scala.concurrent.Await
import akka.actor.{Actor, ActorSelection, Props}
@@ -38,6 +40,7 @@ private[spark] class CoarseGrainedExecutorBackend(
executorId: String,
hostPort: String,
cores: Int,
+ userClassPath: Seq[URL],
env: SparkEnv)
extends Actor with ActorLogReceive with ExecutorBackend with Logging {
@@ -49,15 +52,21 @@ private[spark] class CoarseGrainedExecutorBackend(
override def preStart() {
logInfo("Connecting to driver: " + driverUrl)
driver = context.actorSelection(driverUrl)
- driver ! RegisterExecutor(executorId, hostPort, cores)
+ driver ! RegisterExecutor(executorId, hostPort, cores, extractLogUrls)
context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
}
+ def extractLogUrls: Map[String, String] = {
+ val prefix = "SPARK_LOG_URL_"
+ sys.env.filterKeys(_.startsWith(prefix))
+ .map(e => (e._1.substring(prefix.length).toLowerCase, e._2))
+ }
+
override def receiveWithLogging = {
case RegisteredExecutor =>
logInfo("Successfully registered with driver")
val (hostname, _) = Utils.parseHostPort(hostPort)
- executor = new Executor(executorId, hostname, env, isLocal = false)
+ executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false)
case RegisterExecutorFailed(message) =>
logError("Slave registration failed: " + message)
@@ -111,7 +120,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
hostname: String,
cores: Int,
appId: String,
- workerUrl: Option[String]) {
+ workerUrl: Option[String],
+ userClassPath: Seq[URL]) {
SignalLogger.register(log)
@@ -123,7 +133,11 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
val executorConf = new SparkConf
val port = executorConf.getInt("spark.executor.port", 0)
val (fetcher, _) = AkkaUtils.createActorSystem(
- "driverPropsFetcher", hostname, port, executorConf, new SecurityManager(executorConf))
+ "driverPropsFetcher",
+ hostname,
+ port,
+ executorConf,
+ new SecurityManager(executorConf))
val driver = fetcher.actorSelection(driverUrl)
val timeout = AkkaUtils.askTimeout(executorConf)
val fut = Patterns.ask(driver, RetrieveSparkProps, timeout)
@@ -132,7 +146,15 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
fetcher.shutdown()
// Create SparkEnv using properties we fetched from the driver.
- val driverConf = new SparkConf().setAll(props)
+ val driverConf = new SparkConf()
+ for ((key, value) <- props) {
+ // this is required for SSL in standalone mode
+ if (SparkConf.isExecutorStartupConf(key)) {
+ driverConf.setIfMissing(key, value)
+ } else {
+ driverConf.set(key, value)
+ }
+ }
val env = SparkEnv.createExecutorEnv(
driverConf, executorId, hostname, port, cores, isLocal = false)
@@ -144,7 +166,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
val sparkHostPort = hostname + ":" + boundPort
env.actorSystem.actorOf(
Props(classOf[CoarseGrainedExecutorBackend],
- driverUrl, executorId, sparkHostPort, cores, env),
+ driverUrl, executorId, sparkHostPort, cores, userClassPath, env),
name = "Executor")
workerUrl.foreach { url =>
env.actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher")
@@ -154,20 +176,69 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
}
def main(args: Array[String]) {
- args.length match {
- case x if x < 5 =>
- System.err.println(
+ var driverUrl: String = null
+ var executorId: String = null
+ var hostname: String = null
+ var cores: Int = 0
+ var appId: String = null
+ var workerUrl: Option[String] = None
+ val userClassPath = new mutable.ListBuffer[URL]()
+
+ var argv = args.toList
+ while (!argv.isEmpty) {
+ argv match {
+ case ("--driver-url") :: value :: tail =>
+ driverUrl = value
+ argv = tail
+ case ("--executor-id") :: value :: tail =>
+ executorId = value
+ argv = tail
+ case ("--hostname") :: value :: tail =>
+ hostname = value
+ argv = tail
+ case ("--cores") :: value :: tail =>
+ cores = value.toInt
+ argv = tail
+ case ("--app-id") :: value :: tail =>
+ appId = value
+ argv = tail
+ case ("--worker-url") :: value :: tail =>
// Worker url is used in spark standalone mode to enforce fate-sharing with worker
- "Usage: CoarseGrainedExecutorBackend " +
- " [] ")
- System.exit(1)
+ workerUrl = Some(value)
+ argv = tail
+ case ("--user-class-path") :: value :: tail =>
+ userClassPath += new URL(value)
+ argv = tail
+ case Nil =>
+ case tail =>
+ System.err.println(s"Unrecognized options: ${tail.mkString(" ")}")
+ printUsageAndExit()
+ }
+ }
- // NB: These arguments are provided by SparkDeploySchedulerBackend (for standalone mode)
- // and CoarseMesosSchedulerBackend (for mesos mode).
- case 5 =>
- run(args(0), args(1), args(2), args(3).toInt, args(4), None)
- case x if x > 5 =>
- run(args(0), args(1), args(2), args(3).toInt, args(4), Some(args(5)))
+ if (driverUrl == null || executorId == null || hostname == null || cores <= 0 ||
+ appId == null) {
+ printUsageAndExit()
}
+
+ run(driverUrl, executorId, hostname, cores, appId, workerUrl, userClassPath)
}
+
+ private def printUsageAndExit() = {
+ System.err.println(
+ """
+ |"Usage: CoarseGrainedExecutorBackend [options]
+ |
+ | Options are:
+ | --driver-url
+ | --executor-id
+ | --hostname
+ | --cores
+ | --app-id
+ | --worker-url
+ | --user-class-path
+ |""".stripMargin)
+ System.exit(1)
+ }
+
}
diff --git a/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala b/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala
new file mode 100644
index 0000000000000..f7604a321f007
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala
@@ -0,0 +1,35 @@
+/*
+ * 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.executor
+
+import org.apache.spark.{TaskCommitDenied, TaskEndReason}
+
+/**
+ * Exception thrown when a task attempts to commit output to HDFS but is denied by the driver.
+ */
+class CommitDeniedException(
+ msg: String,
+ jobID: Int,
+ splitID: Int,
+ attemptID: Int)
+ extends Exception(msg) {
+
+ def toTaskEndReason: TaskEndReason = new TaskCommitDenied(jobID, splitID, attemptID)
+
+}
+
diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala
index 312bb3a1daaa3..a897e532184ac 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -19,6 +19,7 @@ package org.apache.spark.executor
import java.io.File
import java.lang.management.ManagementFactory
+import java.net.URL
import java.nio.ByteBuffer
import java.util.concurrent._
@@ -33,7 +34,8 @@ 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.{SparkUncaughtExceptionHandler, AkkaUtils, Utils}
+import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader,
+ SparkUncaughtExceptionHandler, AkkaUtils, Utils}
/**
* Spark executor used with Mesos, YARN, and the standalone scheduler.
@@ -43,10 +45,10 @@ private[spark] class Executor(
executorId: String,
executorHostname: String,
env: SparkEnv,
+ userClassPath: Seq[URL] = Nil,
isLocal: Boolean = false)
extends Logging
{
-
logInfo(s"Starting executor ID $executorId on host $executorHostname")
// Application dependencies (added through SparkContext) that we've fetched so far on this node.
@@ -75,6 +77,9 @@ private[spark] class Executor(
Thread.setDefaultUncaughtExceptionHandler(SparkUncaughtExceptionHandler)
}
+ // Start worker thread pool
+ val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker")
+
val executorSource = new ExecutorSource(this, executorId)
if (!isLocal) {
@@ -86,6 +91,12 @@ private[spark] class Executor(
private val executorActor = env.actorSystem.actorOf(
Props(new ExecutorActor(executorId)), "ExecutorActor")
+ // Whether to load classes in user jars before those in Spark jars
+ private val userClassPathFirst: Boolean = {
+ conf.getBoolean("spark.executor.userClassPathFirst",
+ conf.getBoolean("spark.files.userClassPathFirst", false))
+ }
+
// Create our ClassLoader
// do this after SparkEnv creation so can access the SecurityManager
private val urlClassLoader = createClassLoader()
@@ -101,9 +112,6 @@ private[spark] class Executor(
// Limit of bytes for total size of results (default is 1GB)
private val maxResultSize = Utils.getMaxResultSize(conf)
- // Start worker thread pool
- val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker")
-
// Maintains the list of running tasks.
private val runningTasks = new ConcurrentHashMap[Long, TaskRunner]
@@ -250,6 +258,11 @@ private[spark] class Executor(
execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled))
}
+ case cDE: CommitDeniedException => {
+ val reason = cDE.toTaskEndReason
+ execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
+ }
+
case t: Throwable => {
// Attempt to exit cleanly by informing the driver of our failure.
// If anything goes wrong (or this was a fatal exception), we will delegate to
@@ -288,17 +301,23 @@ private[spark] class Executor(
* created by the interpreter to the search path
*/
private def createClassLoader(): MutableURLClassLoader = {
+ // Bootstrap the list of jars with the user class path.
+ val now = System.currentTimeMillis()
+ userClassPath.foreach { url =>
+ currentJars(url.getPath().split("/").last) = now
+ }
+
val currentLoader = Utils.getContextOrSparkClassLoader
// For each of the jars in the jarSet, add them to the class loader.
// We assume each of the files has already been fetched.
- val urls = currentJars.keySet.map { uri =>
+ val urls = userClassPath.toArray ++ currentJars.keySet.map { uri =>
new File(uri.split("/").last).toURI.toURL
- }.toArray
- val userClassPathFirst = conf.getBoolean("spark.files.userClassPathFirst", false)
- userClassPathFirst match {
- case true => new ChildExecutorURLClassLoader(urls, currentLoader)
- case false => new ExecutorURLClassLoader(urls, currentLoader)
+ }
+ if (userClassPathFirst) {
+ new ChildFirstURLClassLoader(urls, currentLoader)
+ } else {
+ new MutableURLClassLoader(urls, currentLoader)
}
}
@@ -310,14 +329,13 @@ private[spark] class Executor(
val classUri = conf.get("spark.repl.class.uri", null)
if (classUri != null) {
logInfo("Using REPL class URI: " + classUri)
- val userClassPathFirst: java.lang.Boolean =
- conf.getBoolean("spark.files.userClassPathFirst", false)
try {
+ val _userClassPathFirst: java.lang.Boolean = userClassPathFirst
val klass = Class.forName("org.apache.spark.repl.ExecutorClassLoader")
.asInstanceOf[Class[_ <: ClassLoader]]
val constructor = klass.getConstructor(classOf[SparkConf], classOf[String],
classOf[ClassLoader], classOf[Boolean])
- constructor.newInstance(conf, classUri, parent, userClassPathFirst)
+ constructor.newInstance(conf, classUri, parent, _userClassPathFirst)
} catch {
case _: ClassNotFoundException =>
logError("Could not find org.apache.spark.repl.ExecutorClassLoader on classpath!")
@@ -344,18 +362,23 @@ private[spark] class Executor(
env.securityManager, hadoopConf, timestamp, useCache = !isLocal)
currentFiles(name) = timestamp
}
- for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) {
- logInfo("Fetching " + name + " with timestamp " + timestamp)
- // Fetch file with useCache mode, close cache for local mode.
- Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf,
- env.securityManager, hadoopConf, timestamp, useCache = !isLocal)
- currentJars(name) = timestamp
- // Add it to our class loader
+ for ((name, timestamp) <- newJars) {
val localName = name.split("/").last
- val url = new File(SparkFiles.getRootDirectory, localName).toURI.toURL
- if (!urlClassLoader.getURLs.contains(url)) {
- logInfo("Adding " + url + " to class loader")
- urlClassLoader.addURL(url)
+ val currentTimeStamp = currentJars.get(name)
+ .orElse(currentJars.get(localName))
+ .getOrElse(-1L)
+ if (currentTimeStamp < timestamp) {
+ logInfo("Fetching " + name + " with timestamp " + timestamp)
+ // Fetch file with useCache mode, close cache for local mode.
+ Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf,
+ env.securityManager, hadoopConf, timestamp, useCache = !isLocal)
+ currentJars(name) = timestamp
+ // Add it to our class loader
+ val url = new File(SparkFiles.getRootDirectory, localName).toURI.toURL
+ if (!urlClassLoader.getURLs.contains(url)) {
+ logInfo("Adding " + url + " to class loader")
+ urlClassLoader.addURL(url)
+ }
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala
deleted file mode 100644
index 218ed7b5d2d39..0000000000000
--- a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.executor
-
-import java.net.{URLClassLoader, URL}
-
-import org.apache.spark.util.ParentClassLoader
-
-/**
- * The addURL method in URLClassLoader is protected. We subclass it to make this accessible.
- * We also make changes so user classes can come before the default classes.
- */
-
-private[spark] trait MutableURLClassLoader extends ClassLoader {
- def addURL(url: URL)
- def getURLs: Array[URL]
-}
-
-private[spark] class ChildExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader)
- extends MutableURLClassLoader {
-
- private object userClassLoader extends URLClassLoader(urls, null){
- override def addURL(url: URL) {
- super.addURL(url)
- }
- override def findClass(name: String): Class[_] = {
- super.findClass(name)
- }
- }
-
- private val parentClassLoader = new ParentClassLoader(parent)
-
- override def findClass(name: String): Class[_] = {
- try {
- userClassLoader.findClass(name)
- } catch {
- case e: ClassNotFoundException => {
- parentClassLoader.loadClass(name)
- }
- }
- }
-
- def addURL(url: URL) {
- userClassLoader.addURL(url)
- }
-
- def getURLs() = {
- userClassLoader.getURLs()
- }
-}
-
-private[spark] class ExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader)
- extends URLClassLoader(urls, parent) with MutableURLClassLoader {
-
- override def addURL(url: URL) {
- super.addURL(url)
- }
-}
-
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 97912c68c5982..07b152651dedf 100644
--- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
+++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
@@ -177,8 +177,8 @@ class TaskMetrics extends Serializable {
* Once https://issues.apache.org/jira/browse/SPARK-5225 is addressed,
* we can store all the different inputMetrics (one per readMethod).
*/
- private[spark] def getInputMetricsForReadMethod(readMethod: DataReadMethod):
- InputMetrics =synchronized {
+ private[spark] def getInputMetricsForReadMethod(
+ readMethod: DataReadMethod): InputMetrics = synchronized {
_inputMetrics match {
case None =>
val metrics = new InputMetrics(readMethod)
@@ -194,18 +194,22 @@ class TaskMetrics extends Serializable {
/**
* Aggregates shuffle read metrics for all registered dependencies into shuffleReadMetrics.
*/
- private[spark] def updateShuffleReadMetrics() = synchronized {
- val merged = new ShuffleReadMetrics()
- for (depMetrics <- depsShuffleReadMetrics) {
- merged.incFetchWaitTime(depMetrics.fetchWaitTime)
- merged.incLocalBlocksFetched(depMetrics.localBlocksFetched)
- merged.incRemoteBlocksFetched(depMetrics.remoteBlocksFetched)
- merged.incRemoteBytesRead(depMetrics.remoteBytesRead)
+ private[spark] def updateShuffleReadMetrics(): Unit = synchronized {
+ if (!depsShuffleReadMetrics.isEmpty) {
+ val merged = new ShuffleReadMetrics()
+ for (depMetrics <- depsShuffleReadMetrics) {
+ merged.incFetchWaitTime(depMetrics.fetchWaitTime)
+ merged.incLocalBlocksFetched(depMetrics.localBlocksFetched)
+ merged.incRemoteBlocksFetched(depMetrics.remoteBlocksFetched)
+ merged.incRemoteBytesRead(depMetrics.remoteBytesRead)
+ merged.incLocalBytesRead(depMetrics.localBytesRead)
+ merged.incRecordsRead(depMetrics.recordsRead)
+ }
+ _shuffleReadMetrics = Some(merged)
}
- _shuffleReadMetrics = Some(merged)
}
- private[spark] def updateInputMetrics() = synchronized {
+ private[spark] def updateInputMetrics(): Unit = synchronized {
inputMetrics.foreach(_.updateBytesRead())
}
}
@@ -242,27 +246,31 @@ object DataWriteMethod extends Enumeration with Serializable {
@DeveloperApi
case class InputMetrics(readMethod: DataReadMethod.Value) {
- private val _bytesRead: AtomicLong = new AtomicLong()
+ /**
+ * This is volatile so that it is visible to the updater thread.
+ */
+ @volatile @transient var bytesReadCallback: Option[() => Long] = None
/**
* Total bytes read.
*/
- def bytesRead: Long = _bytesRead.get()
- @volatile @transient var bytesReadCallback: Option[() => Long] = None
+ private var _bytesRead: Long = _
+ def bytesRead: Long = _bytesRead
+ def incBytesRead(bytes: Long) = _bytesRead += bytes
/**
- * Adds additional bytes read for this read method.
+ * Total records read.
*/
- def addBytesRead(bytes: Long) = {
- _bytesRead.addAndGet(bytes)
- }
+ private var _recordsRead: Long = _
+ def recordsRead: Long = _recordsRead
+ def incRecordsRead(records: Long) = _recordsRead += records
/**
* Invoke the bytesReadCallback and mutate bytesRead.
*/
def updateBytesRead() {
bytesReadCallback.foreach { c =>
- _bytesRead.set(c())
+ _bytesRead = c()
}
}
@@ -287,6 +295,13 @@ case class OutputMetrics(writeMethod: DataWriteMethod.Value) {
private var _bytesWritten: Long = _
def bytesWritten = _bytesWritten
private[spark] def setBytesWritten(value : Long) = _bytesWritten = value
+
+ /**
+ * Total records written
+ */
+ private var _recordsWritten: Long = 0L
+ def recordsWritten = _recordsWritten
+ private[spark] def setRecordsWritten(value: Long) = _recordsWritten = value
}
/**
@@ -301,7 +316,7 @@ class ShuffleReadMetrics extends Serializable {
private var _remoteBlocksFetched: Int = _
def remoteBlocksFetched = _remoteBlocksFetched
private[spark] def incRemoteBlocksFetched(value: Int) = _remoteBlocksFetched += value
- private[spark] def defRemoteBlocksFetched(value: Int) = _remoteBlocksFetched -= value
+ private[spark] def decRemoteBlocksFetched(value: Int) = _remoteBlocksFetched -= value
/**
* Number of local blocks fetched in this shuffle by this task
@@ -309,8 +324,7 @@ class ShuffleReadMetrics extends Serializable {
private var _localBlocksFetched: Int = _
def localBlocksFetched = _localBlocksFetched
private[spark] def incLocalBlocksFetched(value: Int) = _localBlocksFetched += value
- private[spark] def defLocalBlocksFetched(value: Int) = _localBlocksFetched -= value
-
+ private[spark] def decLocalBlocksFetched(value: Int) = _localBlocksFetched -= value
/**
* Time the task spent waiting for remote shuffle blocks. This only includes the time
@@ -330,10 +344,30 @@ class ShuffleReadMetrics extends Serializable {
private[spark] def incRemoteBytesRead(value: Long) = _remoteBytesRead += value
private[spark] def decRemoteBytesRead(value: Long) = _remoteBytesRead -= value
+ /**
+ * Shuffle data that was read from the local disk (as opposed to from a remote executor).
+ */
+ private var _localBytesRead: Long = _
+ def localBytesRead = _localBytesRead
+ private[spark] def incLocalBytesRead(value: Long) = _localBytesRead += value
+
+ /**
+ * Total bytes fetched in the shuffle by this task (both remote and local).
+ */
+ def totalBytesRead = _remoteBytesRead + _localBytesRead
+
/**
* Number of blocks fetched in this shuffle by this task (remote or local)
*/
def totalBlocksFetched = _remoteBlocksFetched + _localBlocksFetched
+
+ /**
+ * Total number of records read from the shuffle by this task
+ */
+ private var _recordsRead: Long = _
+ def recordsRead = _recordsRead
+ private[spark] def incRecordsRead(value: Long) = _recordsRead += value
+ private[spark] def decRecordsRead(value: Long) = _recordsRead -= value
}
/**
@@ -358,5 +392,12 @@ class ShuffleWriteMetrics extends Serializable {
private[spark] def incShuffleWriteTime(value: Long) = _shuffleWriteTime += value
private[spark] def decShuffleWriteTime(value: Long) = _shuffleWriteTime -= value
-
+ /**
+ * Total number of records written to the shuffle by this task
+ */
+ @volatile private var _shuffleRecordsWritten: Long = _
+ def shuffleRecordsWritten = _shuffleRecordsWritten
+ private[spark] def incShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten += value
+ private[spark] def decShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten -= value
+ private[spark] def setShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten = value
}
diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
index f856890d279f4..0709b6d689e86 100644
--- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
+++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
@@ -26,7 +26,6 @@ import org.xerial.snappy.{Snappy, SnappyInputStream, SnappyOutputStream}
import org.apache.spark.SparkConf
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.util.Utils
-import org.apache.spark.Logging
/**
* :: DeveloperApi ::
@@ -53,8 +52,12 @@ private[spark] object CompressionCodec {
"lzf" -> classOf[LZFCompressionCodec].getName,
"snappy" -> classOf[SnappyCompressionCodec].getName)
+ def getCodecName(conf: SparkConf): String = {
+ conf.get(configKey, DEFAULT_COMPRESSION_CODEC)
+ }
+
def createCodec(conf: SparkConf): CompressionCodec = {
- createCodec(conf, conf.get(configKey, DEFAULT_COMPRESSION_CODEC))
+ createCodec(conf, getCodecName(conf))
}
def createCodec(conf: SparkConf, codecName: String): CompressionCodec = {
@@ -71,6 +74,20 @@ private[spark] object CompressionCodec {
s"Consider setting $configKey=$FALLBACK_COMPRESSION_CODEC"))
}
+ /**
+ * Return the short version of the given codec name.
+ * If it is already a short name, just return it.
+ */
+ def getShortName(codecName: String): String = {
+ if (shortCompressionCodecNames.contains(codecName)) {
+ codecName
+ } else {
+ shortCompressionCodecNames
+ .collectFirst { case (k, v) if v == codecName => k }
+ .getOrElse { throw new IllegalArgumentException(s"No short name for codec $codecName.") }
+ }
+ }
+
val FALLBACK_COMPRESSION_CODEC = "lzf"
val DEFAULT_COMPRESSION_CODEC = "snappy"
val ALL_COMPRESSION_CODECS = shortCompressionCodecNames.values.toSeq
diff --git a/core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala b/core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala
new file mode 100644
index 0000000000000..a835012531052
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala
@@ -0,0 +1,25 @@
+/*
+ * 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.launcher
+
+/**
+ * This class makes SparkSubmitOptionParser visible for Spark code outside of the `launcher`
+ * package, since Java doesn't have a feature similar to `private[spark]`, and we don't want
+ * that class to be public.
+ */
+private[spark] abstract class SparkSubmitArgumentsParser extends SparkSubmitOptionParser
diff --git a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala
new file mode 100644
index 0000000000000..9be98723aed14
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala
@@ -0,0 +1,50 @@
+/*
+ * 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.launcher
+
+import java.io.File
+import java.util.{HashMap => JHashMap, List => JList, Map => JMap}
+
+import scala.collection.JavaConversions._
+
+import org.apache.spark.deploy.Command
+
+/**
+ * This class is used by CommandUtils. It uses some package-private APIs in SparkLauncher, and since
+ * Java doesn't have a feature similar to `private[spark]`, and we don't want that class to be
+ * public, needs to live in the same package as the rest of the library.
+ */
+private[spark] class WorkerCommandBuilder(sparkHome: String, memoryMb: Int, command: Command)
+ extends AbstractCommandBuilder {
+
+ childEnv.putAll(command.environment)
+ childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, sparkHome)
+
+ override def buildCommand(env: JMap[String, String]): JList[String] = {
+ val cmd = buildJavaCommand(command.classPathEntries.mkString(File.pathSeparator))
+ cmd.add(s"-Xms${memoryMb}M")
+ cmd.add(s"-Xmx${memoryMb}M")
+ command.javaOpts.foreach(cmd.add)
+ addPermGenSizeOpt(cmd)
+ addOptionString(cmd, getenv("SPARK_JAVA_OPTS"))
+ cmd
+ }
+
+ def buildCommand(): JList[String] = buildCommand(new JHashMap[String, String]())
+
+}
diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala
index 1b7a5d1f1980a..8edf493780687 100644
--- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala
@@ -28,12 +28,12 @@ import org.apache.spark.util.Utils
private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging {
- val DEFAULT_PREFIX = "*"
- val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r
- val METRICS_CONF = "metrics.properties"
+ private val DEFAULT_PREFIX = "*"
+ private val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r
+ private val DEFAULT_METRICS_CONF_FILENAME = "metrics.properties"
- val properties = new Properties()
- var propertyCategories: mutable.HashMap[String, Properties] = null
+ private[metrics] val properties = new Properties()
+ private[metrics] var propertyCategories: mutable.HashMap[String, Properties] = null
private def setDefaultProperties(prop: Properties) {
prop.setProperty("*.sink.servlet.class", "org.apache.spark.metrics.sink.MetricsServlet")
@@ -47,20 +47,22 @@ private[spark] class MetricsConfig(val configFile: Option[String]) extends Loggi
setDefaultProperties(properties)
// If spark.metrics.conf is not set, try to get file in class path
- var is: InputStream = null
- try {
- is = configFile match {
- case Some(f) => new FileInputStream(f)
- case None => Utils.getSparkClassLoader.getResourceAsStream(METRICS_CONF)
+ val isOpt: Option[InputStream] = configFile.map(new FileInputStream(_)).orElse {
+ try {
+ Option(Utils.getSparkClassLoader.getResourceAsStream(DEFAULT_METRICS_CONF_FILENAME))
+ } catch {
+ case e: Exception =>
+ logError("Error loading default configuration file", e)
+ None
}
+ }
- if (is != null) {
+ isOpt.foreach { is =>
+ try {
properties.load(is)
+ } finally {
+ is.close()
}
- } catch {
- case e: Exception => logError("Error loading configure file", e)
- } finally {
- if (is != null) is.close()
}
propertyCategories = subProperties(properties, INSTANCE_REGEX)
diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
index 83e8eb71260eb..345db36630fd5 100644
--- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
@@ -191,7 +191,10 @@ private[spark] class MetricsSystem private (
sinks += sink.asInstanceOf[Sink]
}
} catch {
- case e: Exception => logError("Sink class " + classPath + " cannot be instantialized", e)
+ case e: Exception => {
+ logError("Sink class " + classPath + " cannot be instantialized")
+ throw e
+ }
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/Slf4jSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/Slf4jSink.scala
new file mode 100644
index 0000000000000..e8b3074e8f1a6
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/Slf4jSink.scala
@@ -0,0 +1,68 @@
+/*
+ * 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.metrics.sink
+
+import java.util.Properties
+import java.util.concurrent.TimeUnit
+
+import com.codahale.metrics.{Slf4jReporter, MetricRegistry}
+
+import org.apache.spark.SecurityManager
+import org.apache.spark.metrics.MetricsSystem
+
+private[spark] class Slf4jSink(
+ val property: Properties,
+ val registry: MetricRegistry,
+ securityMgr: SecurityManager)
+ extends Sink {
+ val SLF4J_DEFAULT_PERIOD = 10
+ val SLF4J_DEFAULT_UNIT = "SECONDS"
+
+ val SLF4J_KEY_PERIOD = "period"
+ val SLF4J_KEY_UNIT = "unit"
+
+ val pollPeriod = Option(property.getProperty(SLF4J_KEY_PERIOD)) match {
+ case Some(s) => s.toInt
+ case None => SLF4J_DEFAULT_PERIOD
+ }
+
+ val pollUnit: TimeUnit = Option(property.getProperty(SLF4J_KEY_UNIT)) match {
+ case Some(s) => TimeUnit.valueOf(s.toUpperCase())
+ case None => TimeUnit.valueOf(SLF4J_DEFAULT_UNIT)
+ }
+
+ MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod)
+
+ val reporter: Slf4jReporter = Slf4jReporter.forRegistry(registry)
+ .convertDurationsTo(TimeUnit.MILLISECONDS)
+ .convertRatesTo(TimeUnit.SECONDS)
+ .build()
+
+ override def start() {
+ reporter.start(pollPeriod, pollUnit)
+ }
+
+ override def stop() {
+ reporter.stop()
+ }
+
+ override def report() {
+ reporter.report()
+ }
+}
+
diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala
index 5ad73c3d27f47..b6249b492150a 100644
--- a/core/src/main/scala/org/apache/spark/package.scala
+++ b/core/src/main/scala/org/apache/spark/package.scala
@@ -27,8 +27,7 @@ package org.apache
* contains operations available only on RDDs of Doubles; and
* [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that can
* be saved as SequenceFiles. These operations are automatically available on any RDD of the right
- * type (e.g. RDD[(Int, Int)] through implicit conversions except `saveAsSequenceFile`. You need to
- * `import org.apache.spark.SparkContext._` to make `saveAsSequenceFile` work.
+ * type (e.g. RDD[(Int, Int)] through implicit conversions.
*
* Java programmers should reference the [[org.apache.spark.api.java]] package
* for Spark programming APIs in Java.
diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
index e66f83bb34e30..03afc289736bb 100644
--- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
@@ -213,7 +213,14 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
} else {
basicBucketFunction _
}
- self.mapPartitions(histogramPartition(bucketFunction)).reduce(mergeCounters)
+ if (self.partitions.length == 0) {
+ new Array[Long](buckets.length - 1)
+ } else {
+ // reduce() requires a non-empty RDD. This works because the mapPartitions will make
+ // non-empty partitions out of empty ones. But it doesn't handle the no-partitions case,
+ // which is below
+ self.mapPartitions(histogramPartition(bucketFunction)).reduce(mergeCounters)
+ }
}
}
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 c3e3931042de2..486e86ce1bb19 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -42,10 +42,11 @@ 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.executor.DataReadMethod
import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD
import org.apache.spark.util.{NextIterator, Utils}
import org.apache.spark.scheduler.{HostTaskLocation, HDFSCacheTaskLocation}
+import org.apache.spark.storage.StorageLevel
/**
* A Spark split class that wraps around a Hadoop InputSplit.
@@ -246,7 +247,9 @@ class HadoopRDD[K, V](
case eof: EOFException =>
finished = true
}
-
+ if (!finished) {
+ inputMetrics.incRecordsRead(1)
+ }
(key, value)
}
@@ -260,7 +263,7 @@ class HadoopRDD[K, V](
// If we can't get the bytes read from the FS stats, fall back to the split size,
// which may be inaccurate.
try {
- inputMetrics.addBytesRead(split.inputSplit.value.getLength)
+ inputMetrics.incBytesRead(split.inputSplit.value.getLength)
} catch {
case e: java.io.IOException =>
logWarning("Unable to get input size to set InputMetrics for task", e)
@@ -308,6 +311,15 @@ class HadoopRDD[K, V](
// Do nothing. Hadoop RDD should not be checkpointed.
}
+ override def persist(storageLevel: StorageLevel): this.type = {
+ if (storageLevel.deserialized) {
+ logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" +
+ " behavior because Hadoop's RecordReader reuses the same Writable object for all records." +
+ " Use a map transformation to make copies of the records.")
+ }
+ super.persist(storageLevel)
+ }
+
def getConf: Configuration = getJobConf()
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
index 642a12c1edf6c..e2267861e79df 100644
--- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
@@ -62,11 +62,11 @@ class JdbcRDD[T: ClassTag](
override def getPartitions: Array[Partition] = {
// bounds are inclusive, hence the + 1 here and - 1 on end
- val length = 1 + upperBound - lowerBound
+ val length = BigInt(1) + upperBound - lowerBound
(0 until numPartitions).map(i => {
- val start = lowerBound + ((i * length) / numPartitions).toLong
- val end = lowerBound + (((i + 1) * length) / numPartitions).toLong - 1
- new JdbcPartition(i, start, end)
+ val start = lowerBound + ((i * length) / numPartitions)
+ val end = lowerBound + (((i + 1) * length) / numPartitions) - 1
+ new JdbcPartition(i, start.toLong, end.toLong)
}).toArray
}
@@ -99,21 +99,21 @@ class JdbcRDD[T: ClassTag](
override def close() {
try {
- if (null != rs && ! rs.isClosed()) {
+ if (null != rs) {
rs.close()
}
} catch {
case e: Exception => logWarning("Exception closing resultset", e)
}
try {
- if (null != stmt && ! stmt.isClosed()) {
+ if (null != stmt) {
stmt.close()
}
} catch {
case e: Exception => logWarning("Exception closing statement", e)
}
try {
- if (null != conn && ! conn.isClosed()) {
+ if (null != conn) {
conn.close()
}
logInfo("closed connection")
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 d86f95ac3e485..7fb94840df99c 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -29,16 +29,13 @@ import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, FileSplit}
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.input.WholeTextFileInputFormat
-import org.apache.spark.InterruptibleIterator
-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._
import org.apache.spark.executor.DataReadMethod
import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil
import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD
import org.apache.spark.util.Utils
import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.storage.StorageLevel
private[spark] class NewHadoopPartition(
rddId: Int,
@@ -154,7 +151,9 @@ class NewHadoopRDD[K, V](
throw new java.util.NoSuchElementException("End of stream")
}
havePair = false
-
+ if (!finished) {
+ inputMetrics.incRecordsRead(1)
+ }
(reader.getCurrentKey, reader.getCurrentValue)
}
@@ -168,7 +167,7 @@ class NewHadoopRDD[K, V](
// If we can't get the bytes read from the FS stats, fall back to the split size,
// which may be inaccurate.
try {
- inputMetrics.addBytesRead(split.serializableHadoopSplit.value.getLength)
+ inputMetrics.incBytesRead(split.serializableHadoopSplit.value.getLength)
} catch {
case e: java.io.IOException =>
logWarning("Unable to get input size to set InputMetrics for task", e)
@@ -211,6 +210,16 @@ class NewHadoopRDD[K, V](
locs.getOrElse(split.getLocations.filter(_ != "localhost"))
}
+ override def persist(storageLevel: StorageLevel): this.type = {
+ if (storageLevel.deserialized) {
+ logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" +
+ " behavior because Hadoop's RecordReader reuses the same Writable object for all records." +
+ " Use a map transformation to make copies of the records.")
+ }
+ super.persist(storageLevel)
+ }
+
+
def getConf: Configuration = confBroadcast.value.value
}
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 144f679a59460..6fdfdb734d1b8 100644
--- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
@@ -75,4 +75,27 @@ class OrderedRDDFunctions[K : Ordering : ClassTag,
new ShuffledRDD[K, V, V](self, partitioner).setKeyOrdering(ordering)
}
+ /**
+ * Returns an RDD containing only the elements in the the inclusive range `lower` to `upper`.
+ * If the RDD has been partitioned using a `RangePartitioner`, then this operation can be
+ * performed efficiently by only scanning the partitions that might contain matching elements.
+ * Otherwise, a standard `filter` is applied to all partitions.
+ */
+ def filterByRange(lower: K, upper: K): RDD[P] = {
+
+ def inRange(k: K): Boolean = ordering.gteq(k, lower) && ordering.lteq(k, upper)
+
+ val rddToFilter: RDD[P] = self.partitioner match {
+ case Some(rp: RangePartitioner[K, V]) => {
+ val partitionIndicies = (rp.getPartition(lower), rp.getPartition(upper)) match {
+ case (l, u) => Math.min(l, u) to Math.max(l, u)
+ }
+ PartitionPruningRDD.create(self, partitionIndicies.contains)
+ }
+ case _ =>
+ self
+ }
+ rddToFilter.filter { case (k, v) => inRange(k) }
+ }
+
}
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 49b88a90ab5af..6b4f097ea9ae5 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -34,7 +34,7 @@ import org.apache.hadoop.io.SequenceFile.CompressionType
import org.apache.hadoop.io.compress.CompressionCodec
import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat}
import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewOutputFormat,
-RecordWriter => NewRecordWriter}
+ RecordWriter => NewRecordWriter}
import org.apache.spark._
import org.apache.spark.Partitioner.defaultPartitioner
@@ -993,8 +993,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
val (outputMetrics, bytesWrittenCallback) = initHadoopOutputMetrics(context)
val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]]
+ require(writer != null, "Unable to obtain RecordWriter")
+ var recordsWritten = 0L
try {
- var recordsWritten = 0L
while (iter.hasNext) {
val pair = iter.next()
writer.write(pair._1, pair._2)
@@ -1008,6 +1009,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
}
committer.commitTask(hadoopContext)
bytesWrittenCallback.foreach { fn => outputMetrics.setBytesWritten(fn()) }
+ outputMetrics.setRecordsWritten(recordsWritten)
1
} : Int
@@ -1065,8 +1067,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
writer.setup(context.stageId, context.partitionId, taskAttemptId)
writer.open()
+ var recordsWritten = 0L
try {
- var recordsWritten = 0L
while (iter.hasNext) {
val record = iter.next()
writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef])
@@ -1080,6 +1082,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
}
writer.commit()
bytesWrittenCallback.foreach { fn => outputMetrics.setBytesWritten(fn()) }
+ outputMetrics.setRecordsWritten(recordsWritten)
}
self.context.runJob(self, writeToFile)
@@ -1097,9 +1100,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
private def maybeUpdateOutputMetrics(bytesWrittenCallback: Option[() => Long],
outputMetrics: OutputMetrics, recordsWritten: Long): Unit = {
- if (recordsWritten % PairRDDFunctions.RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES == 0
- && bytesWrittenCallback.isDefined) {
+ if (recordsWritten % PairRDDFunctions.RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES == 0) {
bytesWrittenCallback.foreach { fn => outputMetrics.setBytesWritten(fn()) }
+ outputMetrics.setRecordsWritten(recordsWritten)
}
}
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 97aee58bddbf1..cf0433010aa03 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -25,11 +25,8 @@ import scala.language.implicitConversions
import scala.reflect.{classTag, ClassTag}
import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus
-import org.apache.hadoop.io.BytesWritable
+import org.apache.hadoop.io.{Writable, BytesWritable, NullWritable, Text}
import org.apache.hadoop.io.compress.CompressionCodec
-import org.apache.hadoop.io.NullWritable
-import org.apache.hadoop.io.Text
-import org.apache.hadoop.io.Writable
import org.apache.hadoop.mapred.TextOutputFormat
import org.apache.spark._
@@ -57,8 +54,7 @@ import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, Bernoulli
* [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that
* can be saved as SequenceFiles.
* All operations are automatically available on any RDD of the right type (e.g. RDD[(Int, Int)]
- * through implicit conversions except `saveAsSequenceFile`. You need to
- * `import org.apache.spark.SparkContext._` to make `saveAsSequenceFile` work.
+ * through implicit.
*
* Internally, each RDD is characterized by five main properties:
*
@@ -466,7 +462,13 @@ abstract class RDD[T: ClassTag](
* Return the union of this RDD and another one. Any identical elements will appear multiple
* times (use `.distinct()` to eliminate them).
*/
- def union(other: RDD[T]): RDD[T] = new UnionRDD(sc, Array(this, other))
+ def union(other: RDD[T]): RDD[T] = {
+ if (partitioner.isDefined && other.partitioner == partitioner) {
+ new PartitionerAwareUnionRDD(sc, Array(this, other))
+ } else {
+ new UnionRDD(sc, Array(this, other))
+ }
+ }
/**
* Return the union of this RDD and another one. Any identical elements will appear multiple
@@ -1144,6 +1146,9 @@ abstract class RDD[T: ClassTag](
* Take the first num elements of the RDD. It works by first scanning one partition, and use the
* results from that partition to estimate the number of additional partitions needed to satisfy
* the limit.
+ *
+ * @note due to complications in the internal implementation, this method will raise
+ * an exception if called on an RDD of `Nothing` or `Null`.
*/
def take(num: Int): Array[T] = {
if (num == 0) {
@@ -1256,6 +1261,10 @@ abstract class RDD[T: ClassTag](
def min()(implicit ord: Ordering[T]): T = this.reduce(ord.min)
/**
+ * @note due to complications in the internal implementation, this method will raise an
+ * exception if called on an RDD of `Nothing` or `Null`. This may be come up in practice
+ * because, for example, the type of `parallelize(Seq())` is `RDD[Nothing]`.
+ * (`parallelize(Seq())` should be avoided anyway in favor of `parallelize(Seq[T]())`.)
* @return true if and only if the RDD contains no elements at all. Note that an RDD
* may be empty even when it has at least 1 partition.
*/
@@ -1527,7 +1536,7 @@ abstract class RDD[T: ClassTag](
*/
object RDD {
- // The following implicit functions were in SparkContext before 1.2 and users had to
+ // The following implicit functions were in SparkContext before 1.3 and users had to
// `import SparkContext._` to enable them. Now we move them here to make the compiler find
// them automatically. However, we still keep the old functions in SparkContext for backward
// compatibility and forward to the following functions directly.
@@ -1541,9 +1550,15 @@ object RDD {
new AsyncRDDActions(rdd)
}
- implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag](
- rdd: RDD[(K, V)]): SequenceFileRDDFunctions[K, V] = {
- new SequenceFileRDDFunctions(rdd)
+ implicit def rddToSequenceFileRDDFunctions[K, V](rdd: RDD[(K, V)])
+ (implicit kt: ClassTag[K], vt: ClassTag[V],
+ keyWritableFactory: WritableFactory[K],
+ valueWritableFactory: WritableFactory[V])
+ : SequenceFileRDDFunctions[K, V] = {
+ implicit val keyConverter = keyWritableFactory.convert
+ implicit val valueConverter = valueWritableFactory.convert
+ new SequenceFileRDDFunctions(rdd,
+ keyWritableFactory.writableClass(kt), valueWritableFactory.writableClass(vt))
}
implicit def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag](rdd: RDD[(K, V)])
diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
index 2b48916951430..059f8963691f0 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
@@ -30,13 +30,35 @@ import org.apache.spark.Logging
* through an implicit conversion. Note that this can't be part of PairRDDFunctions because
* we need more implicit parameters to convert our keys and values to Writable.
*
- * Import `org.apache.spark.SparkContext._` at the top of their program to use these functions.
*/
class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag](
- self: RDD[(K, V)])
+ self: RDD[(K, V)],
+ _keyWritableClass: Class[_ <: Writable],
+ _valueWritableClass: Class[_ <: Writable])
extends Logging
with Serializable {
+ @deprecated("It's used to provide backward compatibility for pre 1.3.0.", "1.3.0")
+ def this(self: RDD[(K, V)]) {
+ this(self, null, null)
+ }
+
+ private val keyWritableClass =
+ if (_keyWritableClass == null) {
+ // pre 1.3.0, we need to use Reflection to get the Writable class
+ getWritableClass[K]()
+ } else {
+ _keyWritableClass
+ }
+
+ private val valueWritableClass =
+ if (_valueWritableClass == null) {
+ // pre 1.3.0, we need to use Reflection to get the Writable class
+ getWritableClass[V]()
+ } else {
+ _valueWritableClass
+ }
+
private def getWritableClass[T <% Writable: ClassTag](): Class[_ <: Writable] = {
val c = {
if (classOf[Writable].isAssignableFrom(classTag[T].runtimeClass)) {
@@ -55,6 +77,7 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag
c.asInstanceOf[Class[_ <: Writable]]
}
+
/**
* Output the RDD as a Hadoop SequenceFile using the Writable types we infer from the RDD's key
* and value types. If the key or value are Writable, then we use their classes directly;
@@ -65,26 +88,28 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag
def saveAsSequenceFile(path: String, codec: Option[Class[_ <: CompressionCodec]] = None) {
def anyToWritable[U <% Writable](u: U): Writable = u
- val keyClass = getWritableClass[K]
- val valueClass = getWritableClass[V]
- val convertKey = !classOf[Writable].isAssignableFrom(self.keyClass)
- val convertValue = !classOf[Writable].isAssignableFrom(self.valueClass)
+ // TODO We cannot force the return type of `anyToWritable` be same as keyWritableClass and
+ // valueWritableClass at the compile time. To implement that, we need to add type parameters to
+ // SequenceFileRDDFunctions. however, SequenceFileRDDFunctions is a public class so it will be a
+ // breaking change.
+ val convertKey = self.keyClass != keyWritableClass
+ val convertValue = self.valueClass != valueWritableClass
- logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," +
- valueClass.getSimpleName + ")" )
+ logInfo("Saving as sequence file of type (" + keyWritableClass.getSimpleName + "," +
+ valueWritableClass.getSimpleName + ")" )
val format = classOf[SequenceFileOutputFormat[Writable, Writable]]
val jobConf = new JobConf(self.context.hadoopConfiguration)
if (!convertKey && !convertValue) {
- self.saveAsHadoopFile(path, keyClass, valueClass, format, jobConf, codec)
+ self.saveAsHadoopFile(path, keyWritableClass, valueWritableClass, format, jobConf, codec)
} else if (!convertKey && convertValue) {
self.map(x => (x._1,anyToWritable(x._2))).saveAsHadoopFile(
- path, keyClass, valueClass, format, jobConf, codec)
+ path, keyWritableClass, valueWritableClass, format, jobConf, codec)
} else if (convertKey && !convertValue) {
self.map(x => (anyToWritable(x._1),x._2)).saveAsHadoopFile(
- path, keyClass, valueClass, format, jobConf, codec)
+ path, keyWritableClass, valueWritableClass, format, jobConf, codec)
} else if (convertKey && convertValue) {
self.map(x => (anyToWritable(x._1),anyToWritable(x._2))).saveAsHadoopFile(
- path, keyClass, valueClass, format, jobConf, codec)
+ path, keyWritableClass, valueWritableClass, format, jobConf, codec)
}
}
}
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 1cfe98673773a..e4170a55b7981 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -26,7 +26,6 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack}
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.language.postfixOps
-import scala.reflect.ClassTag
import scala.util.control.NonFatal
import akka.pattern.ask
@@ -38,7 +37,7 @@ import org.apache.spark.executor.TaskMetrics
import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult}
import org.apache.spark.rdd.RDD
import org.apache.spark.storage._
-import org.apache.spark.util.{CallSite, EventLoop, SystemClock, Clock, Utils}
+import org.apache.spark.util._
import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat
/**
@@ -63,7 +62,7 @@ class DAGScheduler(
mapOutputTracker: MapOutputTrackerMaster,
blockManagerMaster: BlockManagerMaster,
env: SparkEnv,
- clock: Clock = SystemClock)
+ clock: Clock = new SystemClock())
extends Logging {
def this(sc: SparkContext, taskScheduler: TaskScheduler) = {
@@ -98,7 +97,13 @@ class DAGScheduler(
private[scheduler] val activeJobs = new HashSet[ActiveJob]
- // Contains the locations that each RDD's partitions are cached on
+ /**
+ * Contains the locations that each RDD's partitions are cached on. This map's keys are RDD ids
+ * and its values are arrays indexed by partition numbers. Each array value is the set of
+ * locations where that RDD partition is cached.
+ *
+ * All accesses to this map should be guarded by synchronizing on it (see SPARK-4454).
+ */
private val cacheLocs = new HashMap[Int, Array[Seq[TaskLocation]]]
// For tracking failed nodes, we use the MapOutputTracker's epoch number, which is sent with
@@ -126,6 +131,8 @@ class DAGScheduler(
private[scheduler] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this)
taskScheduler.setDAGScheduler(this)
+ private val outputCommitCoordinator = env.outputCommitCoordinator
+
// Called by TaskScheduler to report task's starting.
def taskStarted(task: Task[_], taskInfo: TaskInfo) {
eventProcessLoop.post(BeginEvent(task, taskInfo))
@@ -181,7 +188,8 @@ class DAGScheduler(
eventProcessLoop.post(TaskSetFailed(taskSet, reason))
}
- private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = {
+ private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = cacheLocs.synchronized {
+ // Note: this doesn't use `getOrElse()` because this method is called O(num tasks) times
if (!cacheLocs.contains(rdd.id)) {
val blockIds = rdd.partitions.indices.map(index => RDDBlockId(rdd.id, index)).toArray[BlockId]
val locs = BlockManager.blockIdsToBlockManagers(blockIds, env, blockManagerMaster)
@@ -192,7 +200,7 @@ class DAGScheduler(
cacheLocs(rdd.id)
}
- private def clearCacheLocs() {
+ private def clearCacheLocs(): Unit = cacheLocs.synchronized {
cacheLocs.clear()
}
@@ -488,7 +496,7 @@ class DAGScheduler(
waiter
}
- def runJob[T, U: ClassTag](
+ def runJob[T, U](
rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
@@ -648,7 +656,7 @@ class DAGScheduler(
// completion events or stage abort
stageIdToStage -= s.id
jobIdToStageIds -= job.jobId
- listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), jobResult))
+ listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), jobResult))
}
}
@@ -697,7 +705,7 @@ class DAGScheduler(
stage.latestInfo.stageFailed(stageFailedMessage)
listenerBus.post(SparkListenerStageCompleted(stage.latestInfo))
}
- listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), JobFailed(error)))
+ listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), JobFailed(error)))
}
}
@@ -736,7 +744,7 @@ class DAGScheduler(
logInfo("Missing parents: " + getMissingParentStages(finalStage))
val shouldRunLocally =
localExecutionEnabled && allowLocal && finalStage.parents.isEmpty && partitions.length == 1
- val jobSubmissionTime = clock.getTime()
+ val jobSubmissionTime = clock.getTimeMillis()
if (shouldRunLocally) {
// Compute very short actions like first() or take() with no parent stages locally.
listenerBus.post(
@@ -808,6 +816,7 @@ class DAGScheduler(
// will be posted, which should always come after a corresponding SparkListenerStageSubmitted
// event.
stage.latestInfo = StageInfo.fromStage(stage, Some(partitionsToCompute.size))
+ outputCommitCoordinator.stageStart(stage.id)
listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties))
// TODO: Maybe we can keep the taskBinary in Stage to avoid serializing it multiple times.
@@ -861,10 +870,11 @@ class DAGScheduler(
logDebug("New pending tasks: " + stage.pendingTasks)
taskScheduler.submitTasks(
new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties))
- stage.latestInfo.submissionTime = Some(clock.getTime())
+ stage.latestInfo.submissionTime = Some(clock.getTimeMillis())
} else {
// Because we posted SparkListenerStageSubmitted earlier, we should post
// SparkListenerStageCompleted here in case there are no tasks to run.
+ outputCommitCoordinator.stageEnd(stage.id)
listenerBus.post(SparkListenerStageCompleted(stage.latestInfo))
logDebug("Stage " + stage + " is actually done; %b %d %d".format(
stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions))
@@ -879,8 +889,16 @@ class DAGScheduler(
if (event.accumUpdates != null) {
try {
Accumulators.add(event.accumUpdates)
+
event.accumUpdates.foreach { case (id, partialValue) =>
- val acc = Accumulators.originals(id).asInstanceOf[Accumulable[Any, Any]]
+ // In this instance, although the reference in Accumulators.originals is a WeakRef,
+ // it's guaranteed to exist since the event.accumUpdates Map exists
+
+ val acc = Accumulators.originals(id).get match {
+ case Some(accum) => accum.asInstanceOf[Accumulable[Any, Any]]
+ case None => throw new NullPointerException("Non-existent reference to Accumulator")
+ }
+
// To avoid UI cruft, ignore cases where value wasn't updated
if (acc.name.isDefined && partialValue != acc.zero) {
val name = acc.name.get
@@ -909,6 +927,9 @@ class DAGScheduler(
val stageId = task.stageId
val taskType = Utils.getFormattedClassName(task)
+ outputCommitCoordinator.taskCompleted(stageId, task.partitionId,
+ event.taskInfo.attempt, event.reason)
+
// The success case is dealt with separately below, since we need to compute accumulator
// updates before posting.
if (event.reason != Success) {
@@ -921,16 +942,17 @@ class DAGScheduler(
// Skip all the actions if the stage has been cancelled.
return
}
+
val stage = stageIdToStage(task.stageId)
def markStageAsFinished(stage: Stage, errorMessage: Option[String] = None) = {
val serviceTime = stage.latestInfo.submissionTime match {
- case Some(t) => "%.03f".format((clock.getTime() - t) / 1000.0)
+ case Some(t) => "%.03f".format((clock.getTimeMillis() - t) / 1000.0)
case _ => "Unknown"
}
if (errorMessage.isEmpty) {
logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime))
- stage.latestInfo.completionTime = Some(clock.getTime())
+ stage.latestInfo.completionTime = Some(clock.getTimeMillis())
} else {
stage.latestInfo.stageFailed(errorMessage.get)
logInfo("%s (%s) failed in %s s".format(stage, stage.name, serviceTime))
@@ -956,7 +978,7 @@ class DAGScheduler(
markStageAsFinished(stage)
cleanupStateForJobAndIndependentStages(job)
listenerBus.post(
- SparkListenerJobEnd(job.jobId, clock.getTime(), JobSucceeded))
+ SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), JobSucceeded))
}
// taskSucceeded runs some user code that might throw an exception. Make sure
@@ -1073,6 +1095,9 @@ class DAGScheduler(
handleExecutorLost(bmAddress.executorId, fetchFailed = true, Some(task.epoch))
}
+ case commitDenied: TaskCommitDenied =>
+ // Do nothing here, left up to the TaskScheduler to decide how to handle denied commits
+
case ExceptionFailure(className, description, stackTrace, fullStackTrace, metrics) =>
// Do nothing here, left up to the TaskScheduler to decide how to handle user failures
@@ -1169,7 +1194,7 @@ class DAGScheduler(
}
val dependentJobs: Seq[ActiveJob] =
activeJobs.filter(job => stageDependsOn(job.finalStage, failedStage)).toSeq
- failedStage.latestInfo.completionTime = Some(clock.getTime())
+ failedStage.latestInfo.completionTime = Some(clock.getTimeMillis())
for (job <- dependentJobs) {
failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason")
}
@@ -1224,7 +1249,7 @@ class DAGScheduler(
if (ableToCancelStages) {
job.listener.jobFailed(error)
cleanupStateForJobAndIndependentStages(job)
- listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), JobFailed(error)))
+ listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), JobFailed(error)))
}
}
@@ -1265,17 +1290,26 @@ class DAGScheduler(
}
/**
- * Synchronized method that might be called from other threads.
+ * Gets the locality information associated with a partition of a particular RDD.
+ *
+ * This method is thread-safe and is called from both DAGScheduler and SparkContext.
+ *
* @param rdd whose partitions are to be looked at
* @param partition to lookup locality information for
* @return list of machines that are preferred by the partition
*/
private[spark]
- def getPreferredLocs(rdd: RDD[_], partition: Int): Seq[TaskLocation] = synchronized {
+ def getPreferredLocs(rdd: RDD[_], partition: Int): Seq[TaskLocation] = {
getPreferredLocsInternal(rdd, partition, new HashSet)
}
- /** Recursive implementation for getPreferredLocs. */
+ /**
+ * Recursive implementation for getPreferredLocs.
+ *
+ * This method is thread-safe because it only accesses DAGScheduler state through thread-safe
+ * methods (getCacheLocs()); please be careful when modifying this method, because any new
+ * DAGScheduler state accessed by it may require additional synchronization.
+ */
private def getPreferredLocsInternal(
rdd: RDD[_],
partition: Int,
diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
index 30075c172bdb1..34fa6d27c3a45 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
@@ -62,6 +62,15 @@ private[spark] class EventLoggingListener(
private val testing = sparkConf.getBoolean("spark.eventLog.testing", false)
private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024
private val fileSystem = Utils.getHadoopFileSystem(new URI(logBaseDir), hadoopConf)
+ private val compressionCodec =
+ if (shouldCompress) {
+ Some(CompressionCodec.createCodec(sparkConf))
+ } else {
+ None
+ }
+ private val compressionCodecName = compressionCodec.map { c =>
+ CompressionCodec.getShortName(c.getClass.getName)
+ }
// Only defined if the file system scheme is not local
private var hadoopDataStream: Option[FSDataOutputStream] = None
@@ -80,13 +89,13 @@ private[spark] class EventLoggingListener(
private[scheduler] val loggedEvents = new ArrayBuffer[JValue]
// Visible for tests only.
- private[scheduler] val logPath = getLogPath(logBaseDir, appId)
+ private[scheduler] val logPath = getLogPath(logBaseDir, appId, compressionCodecName)
/**
* Creates the log file in the configured log directory.
*/
def start() {
- if (!fileSystem.isDirectory(new Path(logBaseDir))) {
+ if (!fileSystem.getFileStatus(new Path(logBaseDir)).isDir) {
throw new IllegalArgumentException(s"Log directory $logBaseDir does not exist.")
}
@@ -111,19 +120,19 @@ private[spark] class EventLoggingListener(
hadoopDataStream.get
}
- val compressionCodec =
- if (shouldCompress) {
- Some(CompressionCodec.createCodec(sparkConf))
- } else {
- None
- }
-
- fileSystem.setPermission(path, LOG_FILE_PERMISSIONS)
- val logStream = initEventLog(new BufferedOutputStream(dstream, outputBufferSize),
- compressionCodec)
- writer = Some(new PrintWriter(logStream))
+ try {
+ val cstream = compressionCodec.map(_.compressedOutputStream(dstream)).getOrElse(dstream)
+ val bstream = new BufferedOutputStream(cstream, outputBufferSize)
- logInfo("Logging events to %s".format(logPath))
+ EventLoggingListener.initEventLog(bstream)
+ fileSystem.setPermission(path, LOG_FILE_PERMISSIONS)
+ writer = Some(new PrintWriter(bstream))
+ logInfo("Logging events to %s".format(logPath))
+ } catch {
+ case e: Exception =>
+ dstream.close()
+ throw e
+ }
}
/** Log the event as JSON. */
@@ -201,77 +210,57 @@ private[spark] object EventLoggingListener extends Logging {
// Suffix applied to the names of files still being written by applications.
val IN_PROGRESS = ".inprogress"
val DEFAULT_LOG_DIR = "/tmp/spark-events"
+ val SPARK_VERSION_KEY = "SPARK_VERSION"
+ val COMPRESSION_CODEC_KEY = "COMPRESSION_CODEC"
private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
- // Marker for the end of header data in a log file. After this marker, log data, potentially
- // compressed, will be found.
- private val HEADER_END_MARKER = "=== LOG_HEADER_END ==="
-
- // To avoid corrupted files causing the heap to fill up. Value is arbitrary.
- private val MAX_HEADER_LINE_LENGTH = 4096
-
// A cache for compression codecs to avoid creating the same codec many times
private val codecMap = new mutable.HashMap[String, CompressionCodec]
/**
- * Write metadata about the event log to the given stream.
- *
- * The header is a serialized version of a map, except it does not use Java serialization to
- * avoid incompatibilities between different JDKs. It writes one map entry per line, in
- * "key=value" format.
+ * Write metadata about an event log to the given stream.
+ * The metadata is encoded in the first line of the event log as JSON.
*
- * The very last entry in the header is the `HEADER_END_MARKER` marker, so that the parsing code
- * can know when to stop.
- *
- * The format needs to be kept in sync with the openEventLog() method below. Also, it cannot
- * change in new Spark versions without some other way of detecting the change (like some
- * metadata encoded in the file name).
- *
- * @param logStream Raw output stream to the even log file.
- * @param compressionCodec Optional compression codec to use.
- * @return A stream where to write event log data. This may be a wrapper around the original
- * stream (for example, when compression is enabled).
+ * @param logStream Raw output stream to the event log file.
*/
- def initEventLog(
- logStream: OutputStream,
- compressionCodec: Option[CompressionCodec]): OutputStream = {
- val meta = mutable.HashMap(("version" -> SPARK_VERSION))
- compressionCodec.foreach { codec =>
- meta += ("compressionCodec" -> codec.getClass().getName())
- }
-
- def write(entry: String) = {
- val bytes = entry.getBytes(Charsets.UTF_8)
- if (bytes.length > MAX_HEADER_LINE_LENGTH) {
- throw new IOException(s"Header entry too long: ${entry}")
- }
- logStream.write(bytes, 0, bytes.length)
- }
-
- meta.foreach { case (k, v) => write(s"$k=$v\n") }
- write(s"$HEADER_END_MARKER\n")
- compressionCodec.map(_.compressedOutputStream(logStream)).getOrElse(logStream)
+ def initEventLog(logStream: OutputStream): Unit = {
+ val metadata = SparkListenerLogStart(SPARK_VERSION)
+ val metadataJson = compact(JsonProtocol.logStartToJson(metadata)) + "\n"
+ logStream.write(metadataJson.getBytes(Charsets.UTF_8))
}
/**
* Return a file-system-safe path to the log file for the given application.
*
+ * Note that because we currently only create a single log file for each application,
+ * we must encode all the information needed to parse this event log in the file name
+ * instead of within the file itself. Otherwise, if the file is compressed, for instance,
+ * we won't know which codec to use to decompress the metadata needed to open the file in
+ * the first place.
+ *
* @param logBaseDir Directory where the log file will be written.
* @param appId A unique app ID.
+ * @param compressionCodecName Name to identify the codec used to compress the contents
+ * of the log, or None if compression is not enabled.
* @return A path which consists of file-system-safe characters.
*/
- def getLogPath(logBaseDir: String, appId: String): String = {
- val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase
- Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/")
+ def getLogPath(
+ logBaseDir: String,
+ appId: String,
+ compressionCodecName: Option[String] = None): String = {
+ val sanitizedAppId = appId.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase
+ // e.g. app_123, app_123.lzf
+ val logName = sanitizedAppId + compressionCodecName.map { "." + _ }.getOrElse("")
+ Utils.resolveURI(logBaseDir).toString.stripSuffix("/") + "/" + logName
}
/**
- * Opens an event log file and returns an input stream to the event data.
+ * Opens an event log file and returns an input stream that contains the event data.
*
- * @return 2-tuple (event input stream, Spark version of event data)
+ * @return input stream that holds one JSON record per line.
*/
- def openEventLog(log: Path, fs: FileSystem): (InputStream, String) = {
+ def openEventLog(log: Path, fs: FileSystem): InputStream = {
// It's not clear whether FileSystem.open() throws FileNotFoundException or just plain
// IOException when a file does not exist, so try our best to throw a proper exception.
if (!fs.exists(log)) {
@@ -279,52 +268,17 @@ private[spark] object EventLoggingListener extends Logging {
}
val in = new BufferedInputStream(fs.open(log))
- // Read a single line from the input stream without buffering.
- // We cannot use BufferedReader because we must avoid reading
- // beyond the end of the header, after which the content of the
- // file may be compressed.
- def readLine(): String = {
- val bytes = new ByteArrayOutputStream()
- var next = in.read()
- var count = 0
- while (next != '\n') {
- if (next == -1) {
- throw new IOException("Unexpected end of file.")
- }
- bytes.write(next)
- count = count + 1
- if (count > MAX_HEADER_LINE_LENGTH) {
- throw new IOException("Maximum header line length exceeded.")
- }
- next = in.read()
- }
- new String(bytes.toByteArray(), Charsets.UTF_8)
+
+ // Compression codec is encoded as an extension, e.g. app_123.lzf
+ // Since we sanitize the app ID to not include periods, it is safe to split on it
+ val logName = log.getName.stripSuffix(IN_PROGRESS)
+ val codecName: Option[String] = logName.split("\\.").tail.lastOption
+ val codec = codecName.map { c =>
+ codecMap.getOrElseUpdate(c, CompressionCodec.createCodec(new SparkConf, c))
}
- // Parse the header metadata in the form of k=v pairs
- // This assumes that every line before the header end marker follows this format
try {
- val meta = new mutable.HashMap[String, String]()
- var foundEndMarker = false
- while (!foundEndMarker) {
- readLine() match {
- case HEADER_END_MARKER =>
- foundEndMarker = true
- case entry =>
- val prop = entry.split("=", 2)
- if (prop.length != 2) {
- throw new IllegalArgumentException("Invalid metadata in log file.")
- }
- meta += (prop(0) -> prop(1))
- }
- }
-
- val sparkVersion = meta.get("version").getOrElse(
- throw new IllegalArgumentException("Missing Spark version in log metadata."))
- val codec = meta.get("compressionCodec").map { codecName =>
- codecMap.getOrElseUpdate(codecName, CompressionCodec.createCodec(new SparkConf, codecName))
- }
- (codec.map(_.compressedInputStream(in)).getOrElse(in), sparkVersion)
+ codec.map(_.compressedInputStream(in)).getOrElse(in)
} catch {
case e: Exception =>
in.close()
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 3bb54855bae44..8aa528ac573d0 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
@@ -169,7 +169,8 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener
" BLOCK_FETCHED_LOCAL=" + metrics.localBlocksFetched +
" BLOCK_FETCHED_REMOTE=" + metrics.remoteBlocksFetched +
" REMOTE_FETCH_WAIT_TIME=" + metrics.fetchWaitTime +
- " REMOTE_BYTES_READ=" + metrics.remoteBytesRead
+ " REMOTE_BYTES_READ=" + metrics.remoteBytesRead +
+ " LOCAL_BYTES_READ=" + metrics.localBytesRead
case None => ""
}
val writeMetrics = taskMetrics.shuffleWriteMetrics match {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala
new file mode 100644
index 0000000000000..759df023a6dcf
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala
@@ -0,0 +1,172 @@
+/*
+ * 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.scheduler
+
+import scala.collection.mutable
+
+import akka.actor.{ActorRef, Actor}
+
+import org.apache.spark._
+import org.apache.spark.util.{AkkaUtils, ActorLogReceive}
+
+private sealed trait OutputCommitCoordinationMessage extends Serializable
+
+private case object StopCoordinator extends OutputCommitCoordinationMessage
+private case class AskPermissionToCommitOutput(stage: Int, task: Long, taskAttempt: Long)
+
+/**
+ * Authority that decides whether tasks can commit output to HDFS. Uses a "first committer wins"
+ * policy.
+ *
+ * OutputCommitCoordinator is instantiated in both the drivers and executors. On executors, it is
+ * configured with a reference to the driver's OutputCommitCoordinatorActor, so requests to commit
+ * output will be forwarded to the driver's OutputCommitCoordinator.
+ *
+ * This class was introduced in SPARK-4879; see that JIRA issue (and the associated pull requests)
+ * for an extensive design discussion.
+ */
+private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging {
+
+ // Initialized by SparkEnv
+ var coordinatorActor: Option[ActorRef] = None
+ private val timeout = AkkaUtils.askTimeout(conf)
+ private val maxAttempts = AkkaUtils.numRetries(conf)
+ private val retryInterval = AkkaUtils.retryWaitMs(conf)
+
+ private type StageId = Int
+ private type PartitionId = Long
+ private type TaskAttemptId = Long
+
+ /**
+ * Map from active stages's id => partition id => task attempt with exclusive lock on committing
+ * output for that partition.
+ *
+ * Entries are added to the top-level map when stages start and are removed they finish
+ * (either successfully or unsuccessfully).
+ *
+ * Access to this map should be guarded by synchronizing on the OutputCommitCoordinator instance.
+ */
+ private val authorizedCommittersByStage: CommittersByStageMap = mutable.Map()
+ private type CommittersByStageMap = mutable.Map[StageId, mutable.Map[PartitionId, TaskAttemptId]]
+
+ /**
+ * Called by tasks to ask whether they can commit their output to HDFS.
+ *
+ * If a task attempt has been authorized to commit, then all other attempts to commit the same
+ * task will be denied. If the authorized task attempt fails (e.g. due to its executor being
+ * lost), then a subsequent task attempt may be authorized to commit its output.
+ *
+ * @param stage the stage number
+ * @param partition the partition number
+ * @param attempt a unique identifier for this task attempt
+ * @return true if this task is authorized to commit, false otherwise
+ */
+ def canCommit(
+ stage: StageId,
+ partition: PartitionId,
+ attempt: TaskAttemptId): Boolean = {
+ val msg = AskPermissionToCommitOutput(stage, partition, attempt)
+ coordinatorActor match {
+ case Some(actor) =>
+ AkkaUtils.askWithReply[Boolean](msg, actor, maxAttempts, retryInterval, timeout)
+ case None =>
+ logError(
+ "canCommit called after coordinator was stopped (is SparkEnv shutdown in progress)?")
+ false
+ }
+ }
+
+ // Called by DAGScheduler
+ private[scheduler] def stageStart(stage: StageId): Unit = synchronized {
+ authorizedCommittersByStage(stage) = mutable.HashMap[PartitionId, TaskAttemptId]()
+ }
+
+ // Called by DAGScheduler
+ private[scheduler] def stageEnd(stage: StageId): Unit = synchronized {
+ authorizedCommittersByStage.remove(stage)
+ }
+
+ // Called by DAGScheduler
+ private[scheduler] def taskCompleted(
+ stage: StageId,
+ partition: PartitionId,
+ attempt: TaskAttemptId,
+ reason: TaskEndReason): Unit = synchronized {
+ val authorizedCommitters = authorizedCommittersByStage.getOrElse(stage, {
+ logDebug(s"Ignoring task completion for completed stage")
+ return
+ })
+ reason match {
+ case Success =>
+ // The task output has been committed successfully
+ case denied: TaskCommitDenied =>
+ logInfo(
+ s"Task was denied committing, stage: $stage, partition: $partition, attempt: $attempt")
+ case otherReason =>
+ logDebug(s"Authorized committer $attempt (stage=$stage, partition=$partition) failed;" +
+ s" clearing lock")
+ authorizedCommitters.remove(partition)
+ }
+ }
+
+ def stop(): Unit = synchronized {
+ coordinatorActor.foreach(_ ! StopCoordinator)
+ coordinatorActor = None
+ authorizedCommittersByStage.clear()
+ }
+
+ // Marked private[scheduler] instead of private so this can be mocked in tests
+ private[scheduler] def handleAskPermissionToCommit(
+ stage: StageId,
+ partition: PartitionId,
+ attempt: TaskAttemptId): Boolean = synchronized {
+ authorizedCommittersByStage.get(stage) match {
+ case Some(authorizedCommitters) =>
+ authorizedCommitters.get(partition) match {
+ case Some(existingCommitter) =>
+ logDebug(s"Denying $attempt to commit for stage=$stage, partition=$partition; " +
+ s"existingCommitter = $existingCommitter")
+ false
+ case None =>
+ logDebug(s"Authorizing $attempt to commit for stage=$stage, partition=$partition")
+ authorizedCommitters(partition) = attempt
+ true
+ }
+ case None =>
+ logDebug(s"Stage $stage has completed, so not allowing task attempt $attempt to commit")
+ false
+ }
+ }
+}
+
+private[spark] object OutputCommitCoordinator {
+
+ // This actor is used only for RPC
+ class OutputCommitCoordinatorActor(outputCommitCoordinator: OutputCommitCoordinator)
+ extends Actor with ActorLogReceive with Logging {
+
+ override def receiveWithLogging = {
+ case AskPermissionToCommitOutput(stage, partition, taskAttempt) =>
+ sender ! outputCommitCoordinator.handleAskPermissionToCommit(stage, partition, taskAttempt)
+ case StopCoordinator =>
+ logInfo("OutputCommitCoordinator stopped!")
+ context.stop(self)
+ sender ! true
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala
index 584f4e7789d1a..86f357abb8723 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala
@@ -21,6 +21,7 @@ import java.io.{InputStream, IOException}
import scala.io.Source
+import com.fasterxml.jackson.core.JsonParseException
import org.json4s.jackson.JsonMethods._
import org.apache.spark.Logging
@@ -39,22 +40,40 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging {
* error is thrown by this method.
*
* @param logData Stream containing event log data.
- * @param version Spark version that generated the events.
+ * @param sourceName Filename (or other source identifier) from whence @logData is being read
+ * @param maybeTruncated Indicate whether log file might be truncated (some abnormal situations
+ * encountered, log file might not finished writing) or not
*/
- def replay(logData: InputStream, version: String) {
+ def replay(
+ logData: InputStream,
+ sourceName: String,
+ maybeTruncated: Boolean = false): Unit = {
var currentLine: String = null
+ var lineNumber: Int = 1
try {
val lines = Source.fromInputStream(logData).getLines()
- lines.foreach { line =>
- currentLine = line
- postToAll(JsonProtocol.sparkEventFromJson(parse(line)))
+ while (lines.hasNext) {
+ currentLine = lines.next()
+ try {
+ postToAll(JsonProtocol.sparkEventFromJson(parse(currentLine)))
+ } catch {
+ case jpe: JsonParseException =>
+ // We can only ignore exception from last line of the file that might be truncated
+ if (!maybeTruncated || lines.hasNext) {
+ throw jpe
+ } else {
+ logWarning(s"Got JsonParseException from log file $sourceName" +
+ s" at line $lineNumber, the file might not have finished writing cleanly.")
+ }
+ }
+ lineNumber += 1
}
} catch {
case ioe: IOException =>
throw ioe
case e: Exception =>
- logError("Exception in parsing Spark event log.", e)
- logError("Malformed line: %s\n".format(currentLine))
+ logError(s"Exception parsing Spark event log: $sourceName", e)
+ logError(s"Malformed line #$lineNumber: $currentLine\n")
}
}
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 dd28ddb31de1f..52720d48ca67f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -116,6 +116,11 @@ case class SparkListenerApplicationStart(appName: String, appId: Option[String],
@DeveloperApi
case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent
+/**
+ * An internal class that describes the metadata of an event log.
+ * This event is not meant to be posted to listeners downstream.
+ */
+private[spark] case class SparkListenerLogStart(sparkVersion: String) extends SparkListenerEvent
/**
* :: DeveloperApi ::
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
index fe8a19a2c0cb9..61e69ecc08387 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
@@ -58,6 +58,7 @@ private[spark] trait SparkListenerBus extends ListenerBus[SparkListener, SparkLi
listener.onExecutorAdded(executorAdded)
case executorRemoved: SparkListenerExecutorRemoved =>
listener.onExecutorRemoved(executorRemoved)
+ case logStart: SparkListenerLogStart => // ignore event log metadata
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
index 774f3d8cdb275..3938580aeea59 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
@@ -18,6 +18,7 @@
package org.apache.spark.scheduler
import java.nio.ByteBuffer
+import java.util.concurrent.RejectedExecutionException
import scala.language.existentials
import scala.util.control.NonFatal
@@ -95,25 +96,30 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul
def enqueueFailedTask(taskSetManager: TaskSetManager, tid: Long, taskState: TaskState,
serializedData: ByteBuffer) {
var reason : TaskEndReason = UnknownReason
- getTaskResultExecutor.execute(new Runnable {
- override def run(): Unit = Utils.logUncaughtExceptions {
- try {
- if (serializedData != null && serializedData.limit() > 0) {
- reason = serializer.get().deserialize[TaskEndReason](
- serializedData, Utils.getSparkClassLoader)
+ try {
+ getTaskResultExecutor.execute(new Runnable {
+ override def run(): Unit = Utils.logUncaughtExceptions {
+ try {
+ if (serializedData != null && serializedData.limit() > 0) {
+ reason = serializer.get().deserialize[TaskEndReason](
+ serializedData, Utils.getSparkClassLoader)
+ }
+ } catch {
+ case cnd: ClassNotFoundException =>
+ // Log an error but keep going here -- the task failed, so not catastrophic
+ // if we can't deserialize the reason.
+ val loader = Utils.getContextOrSparkClassLoader
+ logError(
+ "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader)
+ case ex: Exception => {}
}
- } catch {
- case cnd: ClassNotFoundException =>
- // Log an error but keep going here -- the task failed, so not catastrophic if we can't
- // deserialize the reason.
- val loader = Utils.getContextOrSparkClassLoader
- logError(
- "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader)
- case ex: Exception => {}
+ scheduler.handleFailedTask(taskSetManager, tid, taskState, reason)
}
- scheduler.handleFailedTask(taskSetManager, tid, taskState, reason)
- }
- })
+ })
+ } catch {
+ case e: RejectedExecutionException if sparkEnv.isStopped =>
+ // ignore it
+ }
}
def stop() {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
index f095915352b17..ed3418676e077 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
@@ -73,5 +73,9 @@ private[spark] trait TaskScheduler {
* @return An application ID
*/
def applicationId(): String = appId
-
+
+ /**
+ * Process a lost executor
+ */
+ def executorLost(executorId: String, reason: ExecutorLossReason): Unit
}
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 33a7aae5d3fcd..7a9cf1c2e7f30 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -158,7 +158,7 @@ private[spark] class TaskSchedulerImpl(
val tasks = taskSet.tasks
logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks")
this.synchronized {
- val manager = new TaskSetManager(this, taskSet, maxTaskFailures)
+ val manager = createTaskSetManager(taskSet, maxTaskFailures)
activeTaskSets(taskSet.id) = manager
schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties)
@@ -180,6 +180,13 @@ private[spark] class TaskSchedulerImpl(
backend.reviveOffers()
}
+ // Label as private[scheduler] to allow tests to swap in different task set managers if necessary
+ private[scheduler] def createTaskSetManager(
+ taskSet: TaskSet,
+ maxTaskFailures: Int): TaskSetManager = {
+ new TaskSetManager(this, taskSet, maxTaskFailures)
+ }
+
override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized {
logInfo("Cancelling stage " + stageId)
activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) =>
@@ -361,7 +368,7 @@ private[spark] class TaskSchedulerImpl(
dagScheduler.executorHeartbeatReceived(execId, metricsWithStageIds, blockManagerId)
}
- def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long) {
+ def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long): Unit = synchronized {
taskSetManager.handleTaskGettingResult(tid)
}
@@ -429,7 +436,7 @@ private[spark] class TaskSchedulerImpl(
}
}
- def executorLost(executorId: String, reason: ExecutorLossReason) {
+ override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {
var failedExecutor: Option[String] = None
synchronized {
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 5c94c6bbcb37b..529237f0d35dc 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -51,7 +51,7 @@ private[spark] class TaskSetManager(
sched: TaskSchedulerImpl,
val taskSet: TaskSet,
val maxTaskFailures: Int,
- clock: Clock = SystemClock)
+ clock: Clock = new SystemClock())
extends Schedulable with Logging {
val conf = sched.sc.conf
@@ -166,7 +166,7 @@ private[spark] class TaskSetManager(
// last launched a task at that level, and move up a level when localityWaits[curLevel] expires.
// We then move down if we manage to launch a "more local" task.
var currentLocalityIndex = 0 // Index of our current locality level in validLocalityLevels
- var lastLaunchTime = clock.getTime() // Time we last launched a task at this level
+ var lastLaunchTime = clock.getTimeMillis() // Time we last launched a task at this level
override def schedulableQueue = null
@@ -281,7 +281,7 @@ private[spark] class TaskSetManager(
val failed = failedExecutors.get(taskId).get
return failed.contains(execId) &&
- clock.getTime() - failed.get(execId).get < EXECUTOR_TASK_BLACKLIST_TIMEOUT
+ clock.getTimeMillis() - failed.get(execId).get < EXECUTOR_TASK_BLACKLIST_TIMEOUT
}
false
@@ -292,7 +292,8 @@ private[spark] class TaskSetManager(
* an attempt running on this host, in case the host is slow. In addition, the task should meet
* the given locality constraint.
*/
- private def dequeueSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value)
+ // Labeled as protected to allow tests to override providing speculative tasks if necessary
+ protected def dequeueSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value)
: Option[(Int, TaskLocality.Value)] =
{
speculatableTasks.retain(index => !successful(index)) // Remove finished tasks from set
@@ -427,7 +428,7 @@ private[spark] class TaskSetManager(
: Option[TaskDescription] =
{
if (!isZombie) {
- val curTime = clock.getTime()
+ val curTime = clock.getTimeMillis()
var allowedLocality = maxLocality
@@ -458,7 +459,7 @@ private[spark] class TaskSetManager(
lastLaunchTime = curTime
}
// Serialize and return the task
- val startTime = clock.getTime()
+ val startTime = clock.getTimeMillis()
val serializedTask: ByteBuffer = try {
Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser)
} catch {
@@ -506,13 +507,64 @@ private[spark] class TaskSetManager(
* Get the level we can launch tasks according to delay scheduling, based on current wait time.
*/
private def getAllowedLocalityLevel(curTime: Long): TaskLocality.TaskLocality = {
- while (curTime - lastLaunchTime >= localityWaits(currentLocalityIndex) &&
- currentLocalityIndex < myLocalityLevels.length - 1)
- {
- // Jump to the next locality level, and remove our waiting time for the current one since
- // we don't want to count it again on the next one
- lastLaunchTime += localityWaits(currentLocalityIndex)
- currentLocalityIndex += 1
+ // Remove the scheduled or finished tasks lazily
+ def tasksNeedToBeScheduledFrom(pendingTaskIds: ArrayBuffer[Int]): Boolean = {
+ var indexOffset = pendingTaskIds.size
+ while (indexOffset > 0) {
+ indexOffset -= 1
+ val index = pendingTaskIds(indexOffset)
+ if (copiesRunning(index) == 0 && !successful(index)) {
+ return true
+ } else {
+ pendingTaskIds.remove(indexOffset)
+ }
+ }
+ false
+ }
+ // Walk through the list of tasks that can be scheduled at each location and returns true
+ // if there are any tasks that still need to be scheduled. Lazily cleans up tasks that have
+ // already been scheduled.
+ def moreTasksToRunIn(pendingTasks: HashMap[String, ArrayBuffer[Int]]): Boolean = {
+ val emptyKeys = new ArrayBuffer[String]
+ val hasTasks = pendingTasks.exists {
+ case (id: String, tasks: ArrayBuffer[Int]) =>
+ if (tasksNeedToBeScheduledFrom(tasks)) {
+ true
+ } else {
+ emptyKeys += id
+ false
+ }
+ }
+ // The key could be executorId, host or rackId
+ emptyKeys.foreach(id => pendingTasks.remove(id))
+ hasTasks
+ }
+
+ while (currentLocalityIndex < myLocalityLevels.length - 1) {
+ val moreTasks = myLocalityLevels(currentLocalityIndex) match {
+ case TaskLocality.PROCESS_LOCAL => moreTasksToRunIn(pendingTasksForExecutor)
+ case TaskLocality.NODE_LOCAL => moreTasksToRunIn(pendingTasksForHost)
+ case TaskLocality.NO_PREF => pendingTasksWithNoPrefs.nonEmpty
+ case TaskLocality.RACK_LOCAL => moreTasksToRunIn(pendingTasksForRack)
+ }
+ if (!moreTasks) {
+ // This is a performance optimization: if there are no more tasks that can
+ // be scheduled at a particular locality level, there is no point in waiting
+ // for the locality wait timeout (SPARK-4939).
+ lastLaunchTime = curTime
+ logDebug(s"No tasks for locality level ${myLocalityLevels(currentLocalityIndex)}, " +
+ s"so moving to locality level ${myLocalityLevels(currentLocalityIndex + 1)}")
+ currentLocalityIndex += 1
+ } else if (curTime - lastLaunchTime >= localityWaits(currentLocalityIndex)) {
+ // Jump to the next locality level, and reset lastLaunchTime so that the next locality
+ // wait timer doesn't immediately expire
+ lastLaunchTime += localityWaits(currentLocalityIndex)
+ currentLocalityIndex += 1
+ logDebug(s"Moving to ${myLocalityLevels(currentLocalityIndex)} after waiting for " +
+ s"${localityWaits(currentLocalityIndex)}ms")
+ } else {
+ return myLocalityLevels(currentLocalityIndex)
+ }
}
myLocalityLevels(currentLocalityIndex)
}
@@ -542,7 +594,7 @@ private[spark] class TaskSetManager(
/**
* Check whether has enough quota to fetch the result with `size` bytes
*/
- def canFetchMoreResults(size: Long): Boolean = synchronized {
+ def canFetchMoreResults(size: Long): Boolean = sched.synchronized {
totalResultSize += size
calculatedTasks += 1
if (maxResultSize > 0 && totalResultSize > maxResultSize) {
@@ -622,7 +674,7 @@ private[spark] class TaskSetManager(
return
}
val key = ef.description
- val now = clock.getTime()
+ val now = clock.getTimeMillis()
val (printFull, dupCount) = {
if (recentExceptions.contains(key)) {
val (dupCount, printTime) = recentExceptions(key)
@@ -654,10 +706,13 @@ private[spark] class TaskSetManager(
}
// always add to failed executors
failedExecutors.getOrElseUpdate(index, new HashMap[String, Long]()).
- put(info.executorId, clock.getTime())
+ put(info.executorId, clock.getTimeMillis())
sched.dagScheduler.taskEnded(tasks(index), reason, null, null, info, taskMetrics)
addPendingTask(index)
- if (!isZombie && state != TaskState.KILLED) {
+ if (!isZombie && state != TaskState.KILLED && !reason.isInstanceOf[TaskCommitDenied]) {
+ // If a task failed because its attempt to commit was denied, do not count this failure
+ // towards failing the stage. This is intended to prevent spurious stage failures in cases
+ // where many speculative tasks are launched and denied to commit.
assert (null != failureReason)
numFailures(index) += 1
if (numFailures(index) >= maxTaskFailures) {
@@ -671,7 +726,7 @@ private[spark] class TaskSetManager(
maybeFinishTaskSet()
}
- def abort(message: String) {
+ def abort(message: String): Unit = sched.synchronized {
// TODO: Kill running tasks if we were not terminated due to a Mesos error
sched.dagScheduler.taskSetFailed(taskSet, message)
isZombie = true
@@ -766,7 +821,7 @@ private[spark] class TaskSetManager(
val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt
logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation)
if (tasksSuccessful >= minFinishedForSpeculation && tasksSuccessful > 0) {
- val time = clock.getTime()
+ val time = clock.getTimeMillis()
val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray
Arrays.sort(durations)
val medianDuration = durations(min((0.5 * tasksSuccessful).round.toInt, durations.size - 1))
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 1da6fe976da5b..9bf74f4be198d 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
@@ -39,7 +39,11 @@ private[spark] object CoarseGrainedClusterMessages {
case class RegisterExecutorFailed(message: String) extends CoarseGrainedClusterMessage
// Executors to driver
- case class RegisterExecutor(executorId: String, hostPort: String, cores: Int)
+ case class RegisterExecutor(
+ executorId: String,
+ hostPort: String,
+ cores: Int,
+ logUrls: Map[String, String])
extends CoarseGrainedClusterMessage {
Utils.checkHostPort(hostPort, "Expected host port")
}
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 103a5c053c289..6f77fa32ce37b 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
@@ -86,7 +86,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste
}
def receiveWithLogging = {
- case RegisterExecutor(executorId, hostPort, cores) =>
+ case RegisterExecutor(executorId, hostPort, cores, logUrls) =>
Utils.checkHostPort(hostPort, "Host port expected " + hostPort)
if (executorDataMap.contains(executorId)) {
sender ! RegisterExecutorFailed("Duplicate executor ID: " + executorId)
@@ -98,7 +98,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste
totalCoreCount.addAndGet(cores)
totalRegisteredExecutors.addAndGet(1)
val (host, _) = Utils.parseHostPort(hostPort)
- val data = new ExecutorData(sender, sender.path.address, host, cores, cores)
+ val data = new ExecutorData(sender, sender.path.address, host, cores, cores, logUrls)
// This must be synchronized because variables mutated
// in this block are read when requesting executors
CoarseGrainedSchedulerBackend.this.synchronized {
@@ -311,9 +311,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste
/**
* Request an additional number of executors from the cluster manager.
- * Return whether the request is acknowledged.
+ * @return whether the request is acknowledged.
*/
final override def requestExecutors(numAdditionalExecutors: Int): Boolean = synchronized {
+ if (numAdditionalExecutors < 0) {
+ throw new IllegalArgumentException(
+ "Attempted to request a negative number of additional executor(s) " +
+ s"$numAdditionalExecutors from the cluster manager. Please specify a positive number!")
+ }
logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager")
logDebug(s"Number of pending executors is now $numPendingExecutors")
numPendingExecutors += numAdditionalExecutors
@@ -322,6 +327,22 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste
doRequestTotalExecutors(newTotal)
}
+ /**
+ * Express a preference to the cluster manager for a given total number of executors. This can
+ * result in canceling pending requests or filing additional requests.
+ * @return whether the request is acknowledged.
+ */
+ final override def requestTotalExecutors(numExecutors: Int): Boolean = synchronized {
+ if (numExecutors < 0) {
+ throw new IllegalArgumentException(
+ "Attempted to request a negative number of executor(s) " +
+ s"$numExecutors from the cluster manager. Please specify a positive number!")
+ }
+ numPendingExecutors =
+ math.max(numExecutors - numExistingExecutors + executorsPendingToRemove.size, 0)
+ doRequestTotalExecutors(numExecutors)
+ }
+
/**
* Request executors from the cluster manager by specifying the total number desired,
* including existing pending and running executors.
@@ -332,7 +353,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste
* insufficient resources to satisfy the first request. We make the assumption here that the
* cluster manager will eventually fulfill all requests when resources free up.
*
- * Return whether the request is acknowledged.
+ * @return whether the request is acknowledged.
*/
protected def doRequestTotalExecutors(requestedTotal: Int): Boolean = false
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala
index eb52ddfb1eab1..5e571efe76720 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala
@@ -33,5 +33,6 @@ private[cluster] class ExecutorData(
val executorAddress: Address,
override val executorHost: String,
var freeCores: Int,
- override val totalCores: Int
-) extends ExecutorInfo(executorHost, totalCores)
+ override val totalCores: Int,
+ override val logUrlMap: Map[String, String]
+) extends ExecutorInfo(executorHost, totalCores, logUrlMap)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala
index b4738e64c9391..7f218566146a1 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala
@@ -25,8 +25,8 @@ import org.apache.spark.annotation.DeveloperApi
@DeveloperApi
class ExecutorInfo(
val executorHost: String,
- val totalCores: Int
-) {
+ val totalCores: Int,
+ val logUrlMap: Map[String, String]) {
def canEqual(other: Any): Boolean = other.isInstanceOf[ExecutorInfo]
@@ -34,12 +34,13 @@ class ExecutorInfo(
case that: ExecutorInfo =>
(that canEqual this) &&
executorHost == that.executorHost &&
- totalCores == that.totalCores
+ totalCores == that.totalCores &&
+ logUrlMap == that.logUrlMap
case _ => false
}
override def hashCode(): Int = {
- val state = Seq(executorHost, totalCores)
+ val state = Seq(executorHost, totalCores, logUrlMap)
state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b)
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
index ee10aa061f4e9..06786a59524e7 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.{Path, FileSystem}
import org.apache.spark.{Logging, SparkContext, SparkEnv}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.scheduler.TaskSchedulerImpl
+import org.apache.spark.util.AkkaUtils
private[spark] class SimrSchedulerBackend(
scheduler: TaskSchedulerImpl,
@@ -38,7 +39,8 @@ private[spark] class SimrSchedulerBackend(
override def start() {
super.start()
- val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format(
+ val driverUrl = AkkaUtils.address(
+ AkkaUtils.protocol(actorSystem),
SparkEnv.driverActorSystemName,
sc.conf.get("spark.driver.host"),
sc.conf.get("spark.driver.port"),
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index 7eb87a564d6f5..ffd4825705755 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -17,11 +17,13 @@
package org.apache.spark.scheduler.cluster
+import java.util.concurrent.Semaphore
+
import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv}
import org.apache.spark.deploy.{ApplicationDescription, Command}
import org.apache.spark.deploy.client.{AppClient, AppClientListener}
import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl}
-import org.apache.spark.util.Utils
+import org.apache.spark.util.{AkkaUtils, Utils}
private[spark] class SparkDeploySchedulerBackend(
scheduler: TaskSchedulerImpl,
@@ -31,28 +33,34 @@ private[spark] class SparkDeploySchedulerBackend(
with AppClientListener
with Logging {
- var client: AppClient = null
- var stopping = false
- var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _
- @volatile var appId: String = _
+ private var client: AppClient = null
+ private var stopping = false
+
+ @volatile var shutdownCallback: SparkDeploySchedulerBackend => Unit = _
+ @volatile private var appId: String = _
- val registrationLock = new Object()
- var registrationDone = false
+ private val registrationBarrier = new Semaphore(0)
- val maxCores = conf.getOption("spark.cores.max").map(_.toInt)
- val totalExpectedCores = maxCores.getOrElse(0)
+ private val maxCores = conf.getOption("spark.cores.max").map(_.toInt)
+ private val totalExpectedCores = maxCores.getOrElse(0)
override def start() {
super.start()
// The endpoint for executors to talk to us
- val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format(
+ val driverUrl = AkkaUtils.address(
+ AkkaUtils.protocol(actorSystem),
SparkEnv.driverActorSystemName,
conf.get("spark.driver.host"),
conf.get("spark.driver.port"),
CoarseGrainedSchedulerBackend.ACTOR_NAME)
- val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{APP_ID}}",
- "{{WORKER_URL}}")
+ val args = Seq(
+ "--driver-url", driverUrl,
+ "--executor-id", "{{EXECUTOR_ID}}",
+ "--hostname", "{{HOSTNAME}}",
+ "--cores", "{{CORES}}",
+ "--app-id", "{{APP_ID}}",
+ "--worker-url", "{{WORKER_URL}}")
val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions")
.map(Utils.splitCommandString).getOrElse(Seq.empty)
val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath")
@@ -77,7 +85,7 @@ private[spark] class SparkDeploySchedulerBackend(
args, sc.executorEnvs, classPathEntries ++ testingClassPath, libraryPathEntries, javaOpts)
val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("")
val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command,
- appUIAddress, sc.eventLogDir)
+ appUIAddress, sc.eventLogDir, sc.eventLogCodec)
client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf)
client.start()
@@ -89,8 +97,10 @@ private[spark] class SparkDeploySchedulerBackend(
stopping = true
super.stop()
client.stop()
- if (shutdownCallback != null) {
- shutdownCallback(this)
+
+ val callback = shutdownCallback
+ if (callback != null) {
+ callback(this)
}
}
@@ -143,18 +153,11 @@ private[spark] class SparkDeploySchedulerBackend(
}
private def waitForRegistration() = {
- registrationLock.synchronized {
- while (!registrationDone) {
- registrationLock.wait()
- }
- }
+ registrationBarrier.acquire()
}
private def notifyContext() = {
- registrationLock.synchronized {
- registrationDone = true
- registrationLock.notifyAll()
- }
+ registrationBarrier.release()
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index 5289661eb896b..90dfe14352a8e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -31,7 +31,7 @@ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTas
import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException}
import org.apache.spark.scheduler.TaskSchedulerImpl
import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
-import org.apache.spark.util.Utils
+import org.apache.spark.util.{Utils, AkkaUtils}
/**
* A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds
@@ -143,7 +143,8 @@ private[spark] class CoarseMesosSchedulerBackend(
}
val command = CommandInfo.newBuilder()
.setEnvironment(environment)
- val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format(
+ val driverUrl = AkkaUtils.address(
+ AkkaUtils.protocol(sc.env.actorSystem),
SparkEnv.driverActorSystemName,
conf.get("spark.driver.host"),
conf.get("spark.driver.port"),
@@ -153,18 +154,25 @@ private[spark] class CoarseMesosSchedulerBackend(
if (uri == null) {
val runScript = new File(executorSparkHome, "./bin/spark-class").getCanonicalPath
command.setValue(
- "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d %s".format(
- prefixEnv, runScript, driverUrl, offer.getSlaveId.getValue,
- offer.getHostname, numCores, appId))
+ "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend"
+ .format(prefixEnv, runScript) +
+ s" --driver-url $driverUrl" +
+ s" --executor-id ${offer.getSlaveId.getValue}" +
+ s" --hostname ${offer.getHostname}" +
+ s" --cores $numCores" +
+ s" --app-id $appId")
} else {
// Grab everything to the first '.'. We'll use that and '*' to
// glob the directory "correctly".
val basename = uri.split('/').last.split('.').head
command.setValue(
- ("cd %s*; %s " +
- "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d %s")
- .format(basename, prefixEnv, driverUrl, offer.getSlaveId.getValue,
- offer.getHostname, numCores, appId))
+ s"cd $basename*; $prefixEnv " +
+ "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend" +
+ s" --driver-url $driverUrl" +
+ s" --executor-id ${offer.getSlaveId.getValue}" +
+ s" --hostname ${offer.getHostname}" +
+ s" --cores $numCores" +
+ s" --app-id $appId")
command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
}
command.build()
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala
index 5101ec8352e79..705116cb13f54 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala
@@ -21,7 +21,7 @@ import org.apache.spark.SparkContext
private[spark] object MemoryUtils {
// These defaults copied from YARN
- val OVERHEAD_FRACTION = 1.07
+ val OVERHEAD_FRACTION = 1.10
val OVERHEAD_MINIMUM = 384
def calculateTotalMemory(sc: SparkContext) = {
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 c3c546be6da15..cfb6592e14aa8 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
@@ -270,7 +270,8 @@ private[spark] class MesosSchedulerBackend(
mesosTasks.foreach { case (slaveId, tasks) =>
slaveIdToWorkerOffer.get(slaveId).foreach(o =>
listenerBus.post(SparkListenerExecutorAdded(System.currentTimeMillis(), slaveId,
- new ExecutorInfo(o.host, o.cores)))
+ // TODO: Add support for log urls for Mesos
+ new ExecutorInfo(o.host, o.cores, Map.empty)))
)
d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters)
}
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 05b6fa54564b7..d95426d918e19 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
@@ -19,6 +19,9 @@ package org.apache.spark.scheduler.local
import java.nio.ByteBuffer
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
import akka.actor.{Actor, ActorRef, Props}
import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState}
@@ -46,6 +49,8 @@ private[spark] class LocalActor(
private val totalCores: Int)
extends Actor with ActorLogReceive with Logging {
+ import context.dispatcher // to use Akka's scheduler.scheduleOnce()
+
private var freeCores = totalCores
private val localExecutorId = SparkContext.DRIVER_IDENTIFIER
@@ -74,11 +79,16 @@ private[spark] class LocalActor(
def reviveOffers() {
val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores))
- for (task <- scheduler.resourceOffers(offers).flatten) {
+ val tasks = scheduler.resourceOffers(offers).flatten
+ for (task <- tasks) {
freeCores -= scheduler.CPUS_PER_TASK
executor.launchTask(executorBackend, taskId = task.taskId, attemptNumber = task.attemptNumber,
task.name, task.serializedTask)
}
+ if (tasks.isEmpty && scheduler.activeTaskSets.nonEmpty) {
+ // Try to reviveOffer after 1 second, because scheduler may wait for locality timeout
+ context.system.scheduler.scheduleOnce(1000 millis, self, ReviveOffers)
+ }
}
}
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 d56e23ce4478a..dc7aa99738c17 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -20,22 +20,23 @@ package org.apache.spark.serializer
import java.io.{EOFException, InputStream, OutputStream}
import java.nio.ByteBuffer
+import scala.reflect.ClassTag
+
import com.esotericsoftware.kryo.{Kryo, KryoException}
import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput}
import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer}
import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator}
+import org.roaringbitmap.{ArrayContainer, BitmapContainer, RoaringArray, RoaringBitmap}
import org.apache.spark._
import org.apache.spark.api.python.PythonBroadcast
import org.apache.spark.broadcast.HttpBroadcast
-import org.apache.spark.network.nio.{PutBlock, GotBlock, GetBlock}
+import org.apache.spark.network.nio.{GetBlock, GotBlock, PutBlock}
import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus}
import org.apache.spark.storage._
import org.apache.spark.util.BoundedPriorityQueue
import org.apache.spark.util.collection.CompactBuffer
-import scala.reflect.ClassTag
-
/**
* A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]].
*
@@ -58,14 +59,6 @@ class KryoSerializer(conf: SparkConf)
private val classesToRegister = conf.get("spark.kryo.classesToRegister", "")
.split(',')
.filter(!_.isEmpty)
- .map { className =>
- try {
- Class.forName(className)
- } catch {
- case e: Exception =>
- throw new SparkException("Failed to load class to register with Kryo", e)
- }
- }
def newKryoOutput() = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize))
@@ -97,7 +90,8 @@ class KryoSerializer(conf: SparkConf)
// Use the default classloader when calling the user registrator.
Thread.currentThread.setContextClassLoader(classLoader)
// Register classes given through spark.kryo.classesToRegister.
- classesToRegister.foreach { clazz => kryo.register(clazz) }
+ classesToRegister
+ .foreach { className => kryo.register(Class.forName(className, true, classLoader)) }
// Allow the user to register their own classes by setting spark.kryo.registrator.
userRegistrator
.map(Class.forName(_, true, classLoader).newInstance().asInstanceOf[KryoRegistrator])
@@ -164,7 +158,13 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ
override def serialize[T: ClassTag](t: T): ByteBuffer = {
output.clear()
- kryo.writeClassAndObject(output, t)
+ try {
+ kryo.writeClassAndObject(output, t)
+ } catch {
+ case e: KryoException if e.getMessage.startsWith("Buffer overflow") =>
+ throw new SparkException(s"Kryo serialization failed: ${e.getMessage}. To avoid this, " +
+ "increase spark.kryoserializer.buffer.max.mb value.")
+ }
ByteBuffer.wrap(output.toBytes)
}
@@ -209,9 +209,17 @@ private[serializer] object KryoSerializer {
classOf[GetBlock],
classOf[CompressedMapStatus],
classOf[HighlyCompressedMapStatus],
+ classOf[RoaringBitmap],
+ classOf[RoaringArray],
+ classOf[RoaringArray.Element],
+ classOf[Array[RoaringArray.Element]],
+ classOf[ArrayContainer],
+ classOf[BitmapContainer],
classOf[CompactBuffer[_]],
classOf[BlockManagerId],
classOf[Array[Byte]],
+ classOf[Array[Short]],
+ classOf[Array[Long]],
classOf[BoundedPriorityQueue[_]],
classOf[SparkConf]
)
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 e3e7434df45b0..7a2c5ae32d98b 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
@@ -86,6 +86,12 @@ private[hash] object BlockStoreShuffleFetcher extends Logging {
context.taskMetrics.updateShuffleReadMetrics()
})
- new InterruptibleIterator[T](context, completionIter)
+ new InterruptibleIterator[T](context, completionIter) {
+ val readMetrics = context.taskMetrics.createShuffleReadMetricsForDependency()
+ override def next(): T = {
+ readMetrics.incRecordsRead(1)
+ delegate.next()
+ }
+ }
}
}
diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala
index 27496c5a289cb..fa2e617762f55 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala
@@ -88,7 +88,10 @@ private[spark] class SortShuffleWriter[K, V, C](
} finally {
// Clean up our sorter, which may have its own intermediate files
if (sorter != null) {
+ val startTime = System.nanoTime()
sorter.stop()
+ context.taskMetrics.shuffleWriteMetrics.foreach(
+ _.incShuffleWriteTime(System.nanoTime - startTime))
sorter = null
}
}
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
index 8bc5a1cd18b64..c8b7763f03fb7 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -53,7 +53,7 @@ private[spark] class BlockResult(
readMethod: DataReadMethod.Value,
bytes: Long) {
val inputMetrics = new InputMetrics(readMethod)
- inputMetrics.addBytesRead(bytes)
+ inputMetrics.incBytesRead(bytes)
}
/**
@@ -1074,7 +1074,7 @@ private[spark] class BlockManager(
* Remove all blocks belonging to the given broadcast.
*/
def removeBroadcast(broadcastId: Long, tellMaster: Boolean): Int = {
- logInfo(s"Removing broadcast $broadcastId")
+ logDebug(s"Removing broadcast $broadcastId")
val blocksToRemove = blockInfo.keys.collect {
case bid @ BroadcastBlockId(`broadcastId`, _) => bid
}
@@ -1086,7 +1086,7 @@ private[spark] class BlockManager(
* Remove a block from both memory and disk.
*/
def removeBlock(blockId: BlockId, tellMaster: Boolean = true): Unit = {
- logInfo(s"Removing block $blockId")
+ logDebug(s"Removing block $blockId")
val info = blockInfo.get(blockId).orNull
if (info != null) {
info.synchronized {
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
index b63c7f191155c..654796f23c96e 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
@@ -61,7 +61,7 @@ class BlockManagerMaster(
tachyonSize: Long): Boolean = {
val res = askDriverWithReply[Boolean](
UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize, tachyonSize))
- logInfo("Updated info of block " + blockId)
+ logDebug(s"Updated info of block $blockId")
res
}
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
index 64133464d8daa..787b0f96bec32 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
@@ -24,7 +24,7 @@ import scala.collection.JavaConversions._
import scala.concurrent.Future
import scala.concurrent.duration._
-import akka.actor.{Actor, ActorRef, Cancellable}
+import akka.actor.{Actor, ActorRef}
import akka.pattern.ask
import org.apache.spark.{Logging, SparkConf, SparkException}
@@ -52,19 +52,6 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
private val akkaTimeout = AkkaUtils.askTimeout(conf)
- val slaveTimeout = conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120 * 1000)
-
- val checkTimeoutInterval = conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000)
-
- var timeoutCheckingTask: Cancellable = null
-
- override def preStart() {
- import context.dispatcher
- timeoutCheckingTask = context.system.scheduler.schedule(0.seconds,
- checkTimeoutInterval.milliseconds, self, ExpireDeadHosts)
- super.preStart()
- }
-
override def receiveWithLogging = {
case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) =>
register(blockManagerId, maxMemSize, slaveActor)
@@ -118,14 +105,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
case StopBlockManagerMaster =>
sender ! true
- if (timeoutCheckingTask != null) {
- timeoutCheckingTask.cancel()
- }
context.stop(self)
- case ExpireDeadHosts =>
- expireDeadHosts()
-
case BlockManagerHeartbeat(blockManagerId) =>
sender ! heartbeatReceived(blockManagerId)
@@ -207,21 +188,6 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
logInfo(s"Removing block manager $blockManagerId")
}
- private def expireDeadHosts() {
- logTrace("Checking for hosts with no recent heart beats in BlockManagerMaster.")
- val now = System.currentTimeMillis()
- val minSeenTime = now - slaveTimeout
- val toRemove = new mutable.HashSet[BlockManagerId]
- for (info <- blockManagerInfo.values) {
- if (info.lastSeenMs < minSeenTime && !info.blockManagerId.isDriver) {
- logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: "
- + (now - info.lastSeenMs) + "ms exceeds " + slaveTimeout + "ms")
- toRemove += info.blockManagerId
- }
- }
- toRemove.foreach(removeBlockManager)
- }
-
private def removeExecutor(execId: String) {
logInfo("Trying to remove executor " + execId + " from BlockManagerMaster.")
blockManagerIdByExecutor.get(execId).foreach(removeBlockManager)
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
index 3f32099d08cc9..48247453edef0 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
@@ -109,6 +109,4 @@ private[spark] object BlockManagerMessages {
extends ToBlockManagerMaster
case class BlockManagerHeartbeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster
-
- case object ExpireDeadHosts extends ToBlockManagerMaster
}
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
index 3198d766fca37..81164178b9e8e 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
@@ -29,7 +29,8 @@ import org.apache.spark.executor.ShuffleWriteMetrics
* appending data to an existing block, and can guarantee atomicity in the case of faults
* as it allows the caller to revert partial writes.
*
- * This interface does not support concurrent writes.
+ * This interface does not support concurrent writes. Also, once the writer has
+ * been opened, it cannot be reopened again.
*/
private[spark] abstract class BlockObjectWriter(val blockId: BlockId) {
@@ -95,6 +96,7 @@ private[spark] class DiskBlockObjectWriter(
private var ts: TimeTrackingOutputStream = null
private var objOut: SerializationStream = null
private var initialized = false
+ private var hasBeenClosed = false
/**
* Cursors used to represent positions in the file.
@@ -115,11 +117,16 @@ private[spark] class DiskBlockObjectWriter(
private var finalPosition: Long = -1
private var reportedPosition = initialPosition
- /** Calling channel.position() to update the write metrics can be a little bit expensive, so we
- * only call it every N writes */
- private var writesSinceMetricsUpdate = 0
+ /**
+ * Keep track of number of records written and also use this to periodically
+ * output bytes written since the latter is expensive to do for each record.
+ */
+ private var numRecordsWritten = 0
override def open(): BlockObjectWriter = {
+ if (hasBeenClosed) {
+ throw new IllegalStateException("Writer already closed. Cannot be reopened.")
+ }
fos = new FileOutputStream(file, true)
ts = new TimeTrackingOutputStream(fos)
channel = fos.getChannel()
@@ -145,6 +152,7 @@ private[spark] class DiskBlockObjectWriter(
ts = null
objOut = null
initialized = false
+ hasBeenClosed = true
}
}
@@ -168,6 +176,7 @@ private[spark] class DiskBlockObjectWriter(
override def revertPartialWritesAndClose() {
try {
writeMetrics.decShuffleBytesWritten(reportedPosition - initialPosition)
+ writeMetrics.decShuffleRecordsWritten(numRecordsWritten)
if (initialized) {
objOut.flush()
@@ -193,12 +202,11 @@ private[spark] class DiskBlockObjectWriter(
}
objOut.writeObject(value)
+ numRecordsWritten += 1
+ writeMetrics.incShuffleRecordsWritten(1)
- if (writesSinceMetricsUpdate == 32) {
- writesSinceMetricsUpdate = 0
+ if (numRecordsWritten % 32 == 0) {
updateBytesWritten()
- } else {
- writesSinceMetricsUpdate += 1
}
}
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 af05eb3ca69ce..12cd8ea3bdf1f 100644
--- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
@@ -17,9 +17,8 @@
package org.apache.spark.storage
+import java.util.UUID
import java.io.{IOException, File}
-import java.text.SimpleDateFormat
-import java.util.{Date, Random, UUID}
import org.apache.spark.{SparkConf, Logging}
import org.apache.spark.executor.ExecutorExitCode
@@ -37,7 +36,6 @@ import org.apache.spark.util.Utils
private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkConf)
extends Logging {
- private val MAX_DIR_CREATION_ATTEMPTS: Int = 10
private[spark]
val subDirsPerLocalDir = blockManager.conf.getInt("spark.diskStore.subDirectories", 64)
@@ -51,7 +49,7 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon
}
private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir))
- addShutdownHook()
+ private val shutdownHook = addShutdownHook()
/** Looks up a file by hashing it into one of our local subdirectories. */
// This method should be kept in sync with
@@ -123,48 +121,42 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon
}
private def createLocalDirs(conf: SparkConf): Array[File] = {
- val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss")
Utils.getOrCreateLocalRootDirs(conf).flatMap { rootDir =>
- var foundLocalDir = false
- var localDir: File = null
- var localDirId: String = null
- var tries = 0
- val rand = new Random()
- while (!foundLocalDir && tries < MAX_DIR_CREATION_ATTEMPTS) {
- tries += 1
- try {
- localDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536))
- localDir = new File(rootDir, s"spark-local-$localDirId")
- if (!localDir.exists) {
- foundLocalDir = localDir.mkdirs()
- }
- } catch {
- case e: Exception =>
- logWarning(s"Attempt $tries to create local dir $localDir failed", e)
- }
- }
- if (!foundLocalDir) {
- logError(s"Failed $MAX_DIR_CREATION_ATTEMPTS attempts to create local dir in $rootDir." +
- " Ignoring this directory.")
- None
- } else {
+ try {
+ val localDir = Utils.createDirectory(rootDir, "blockmgr")
logInfo(s"Created local directory at $localDir")
Some(localDir)
+ } catch {
+ case e: IOException =>
+ logError(s"Failed to create local dir in $rootDir. Ignoring this directory.", e)
+ None
}
}
}
- private def addShutdownHook() {
- Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") {
+ private def addShutdownHook(): Thread = {
+ val shutdownHook = new Thread("delete Spark local dirs") {
override def run(): Unit = Utils.logUncaughtExceptions {
logDebug("Shutdown hook called")
- DiskBlockManager.this.stop()
+ DiskBlockManager.this.doStop()
}
- })
+ }
+ Runtime.getRuntime.addShutdownHook(shutdownHook)
+ shutdownHook
}
/** Cleanup local dirs and stop shuffle sender. */
private[spark] def stop() {
+ // Remove the shutdown hook. It causes memory leaks if we leave it around.
+ try {
+ Runtime.getRuntime.removeShutdownHook(shutdownHook)
+ } catch {
+ case e: IllegalStateException => None
+ }
+ doStop()
+ }
+
+ private def doStop(): Unit = {
// Only perform cleanup if an external service is not serving our shuffle files.
if (!blockManager.externalShuffleServiceEnabled || blockManager.blockManagerId.isDriver) {
localDirs.foreach { localDir =>
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 71305a46bf570..1be860aea63d0 100644
--- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala
@@ -46,6 +46,14 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
// A mapping from thread ID to amount of memory used for unrolling a block (in bytes)
// All accesses of this map are assumed to have manually synchronized on `accountingLock`
private val unrollMemoryMap = mutable.HashMap[Long, Long]()
+ // Same as `unrollMemoryMap`, but for pending unroll memory as defined below.
+ // Pending unroll memory refers to the intermediate memory occupied by a thread
+ // after the unroll but before the actual putting of the block in the cache.
+ // This chunk of memory is expected to be released *as soon as* we finish
+ // caching the corresponding block as opposed to until after the task finishes.
+ // This is only used if a block is successfully unrolled in its entirety in
+ // memory (SPARK-4777).
+ private val pendingUnrollMemoryMap = mutable.HashMap[Long, Long]()
/**
* The amount of space ensured for unrolling values in memory, shared across all cores.
@@ -184,7 +192,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
val entry = entries.remove(blockId)
if (entry != null) {
currentMemory -= entry.size
- logInfo(s"Block $blockId of size ${entry.size} dropped from memory (free $freeMemory)")
+ logDebug(s"Block $blockId of size ${entry.size} dropped from memory (free $freeMemory)")
true
} else {
false
@@ -283,12 +291,16 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
}
} finally {
- // If we return an array, the values returned do not depend on the underlying vector and
- // we can immediately free up space for other threads. Otherwise, if we return an iterator,
- // we release the memory claimed by this thread later on when the task finishes.
+ // If we return an array, the values returned will later be cached in `tryToPut`.
+ // In this case, we should release the memory after we cache the block there.
+ // Otherwise, if we return an iterator, we release the memory reserved here
+ // later when the task finishes.
if (keepUnrolling) {
- val amountToRelease = currentUnrollMemoryForThisThread - previousMemoryReserved
- releaseUnrollMemoryForThisThread(amountToRelease)
+ accountingLock.synchronized {
+ val amountToRelease = currentUnrollMemoryForThisThread - previousMemoryReserved
+ releaseUnrollMemoryForThisThread(amountToRelease)
+ reservePendingUnrollMemoryForThisThread(amountToRelease)
+ }
}
}
}
@@ -353,6 +365,8 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
val droppedBlockStatus = blockManager.dropFromMemory(blockId, data)
droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) }
}
+ // Release the unroll memory used because we no longer need the underlying Array
+ releasePendingUnrollMemoryForThisThread()
}
ResultWithDroppedBlocks(putSuccess, droppedBlocks)
}
@@ -381,7 +395,10 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
}
// Take into account the amount of memory currently occupied by unrolling blocks
- val actualFreeMemory = freeMemory - currentUnrollMemory
+ // and minus the pending unroll memory for that block on current thread.
+ val threadId = Thread.currentThread().getId
+ val actualFreeMemory = freeMemory - currentUnrollMemory +
+ pendingUnrollMemoryMap.getOrElse(threadId, 0L)
if (actualFreeMemory < space) {
val rddToAdd = getRddId(blockIdToAdd)
@@ -468,11 +485,32 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
}
}
+ /**
+ * Reserve the unroll memory of current unroll successful block used by this thread
+ * until actually put the block into memory entry.
+ */
+ def reservePendingUnrollMemoryForThisThread(memory: Long): Unit = {
+ val threadId = Thread.currentThread().getId
+ accountingLock.synchronized {
+ pendingUnrollMemoryMap(threadId) = pendingUnrollMemoryMap.getOrElse(threadId, 0L) + memory
+ }
+ }
+
+ /**
+ * Release pending unroll memory of current unroll successful block used by this thread
+ */
+ def releasePendingUnrollMemoryForThisThread(): Unit = {
+ val threadId = Thread.currentThread().getId
+ accountingLock.synchronized {
+ pendingUnrollMemoryMap.remove(threadId)
+ }
+ }
+
/**
* Return the amount of memory currently occupied for unrolling blocks across all threads.
*/
def currentUnrollMemory: Long = accountingLock.synchronized {
- unrollMemoryMap.values.sum
+ unrollMemoryMap.values.sum + pendingUnrollMemoryMap.values.sum
}
/**
diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
index ab9ee4f0096bf..8f28ef49a8a6f 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
@@ -234,6 +234,7 @@ final class ShuffleBlockFetcherIterator(
try {
val buf = blockManager.getBlockData(blockId)
shuffleMetrics.incLocalBlocksFetched(1)
+ shuffleMetrics.incLocalBytesRead(buf.size)
buf.retain()
results.put(new SuccessFetchResult(blockId, 0, buf))
} catch {
diff --git a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala
index 27ba9e18237b5..67f572e79314d 100644
--- a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala
+++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala
@@ -28,7 +28,6 @@ import org.apache.spark._
* of them will be combined together, showed in one line.
*/
private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging {
-
// Carrige return
val CR = '\r'
// Update period of progress bar, in milliseconds
@@ -121,4 +120,10 @@ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging {
clear()
lastFinishTime = System.currentTimeMillis()
}
+
+ /**
+ * Tear down the timer thread. The timer thread is a GC root, and it retains the entire
+ * SparkContext if it's not terminated.
+ */
+ def stop(): Unit = timer.cancel()
}
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 88fed833f922d..95f254a9ef22a 100644
--- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
@@ -62,19 +62,28 @@ private[spark] object JettyUtils extends Logging {
securityMgr: SecurityManager): HttpServlet = {
new HttpServlet {
override def doGet(request: HttpServletRequest, response: HttpServletResponse) {
- if (securityMgr.checkUIViewPermissions(request.getRemoteUser)) {
- response.setContentType("%s;charset=utf-8".format(servletParams.contentType))
- response.setStatus(HttpServletResponse.SC_OK)
- val result = servletParams.responder(request)
- response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate")
- response.getWriter.println(servletParams.extractFn(result))
- } else {
- response.setStatus(HttpServletResponse.SC_UNAUTHORIZED)
- response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate")
- response.sendError(HttpServletResponse.SC_UNAUTHORIZED,
- "User is not authorized to access this page.")
+ try {
+ if (securityMgr.checkUIViewPermissions(request.getRemoteUser)) {
+ response.setContentType("%s;charset=utf-8".format(servletParams.contentType))
+ response.setStatus(HttpServletResponse.SC_OK)
+ val result = servletParams.responder(request)
+ response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate")
+ response.getWriter.println(servletParams.extractFn(result))
+ } else {
+ response.setStatus(HttpServletResponse.SC_UNAUTHORIZED)
+ response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate")
+ response.sendError(HttpServletResponse.SC_UNAUTHORIZED,
+ "User is not authorized to access this page.")
+ }
+ } catch {
+ case e: IllegalArgumentException =>
+ response.sendError(HttpServletResponse.SC_BAD_REQUEST, e.getMessage)
}
}
+ // SPARK-5983 ensure TRACE is not supported
+ protected override def doTrace(req: HttpServletRequest, res: HttpServletResponse): Unit = {
+ res.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED)
+ }
}
}
@@ -114,6 +123,10 @@ private[spark] object JettyUtils extends Logging {
val newUrl = new URL(new URL(request.getRequestURL.toString), prefixedDestPath).toString
response.sendRedirect(newUrl)
}
+ // SPARK-5983 ensure TRACE is not supported
+ protected override def doTrace(req: HttpServletRequest, res: HttpServletResponse): Unit = {
+ res.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED)
+ }
}
createServletHandler(srcPath, servlet, basePath)
}
diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala
index 4307029d44fbb..cae6870c2ab20 100644
--- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala
+++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala
@@ -29,15 +29,20 @@ private[spark] object ToolTips {
val SHUFFLE_READ_BLOCKED_TIME =
"Time that the task spent blocked waiting for shuffle data to be read from remote machines."
- val INPUT = "Bytes read from Hadoop or from Spark storage."
+ val INPUT = "Bytes and records read from Hadoop or from Spark storage."
- val OUTPUT = "Bytes written to Hadoop."
+ val OUTPUT = "Bytes and records written to Hadoop."
- val SHUFFLE_WRITE = "Bytes written to disk in order to be read by a shuffle in a future stage."
+ val SHUFFLE_WRITE =
+ "Bytes and records written to disk in order to be read by a shuffle in a future stage."
val SHUFFLE_READ =
- """Bytes read from remote executors. Typically less than shuffle write bytes
- because this does not include shuffle data read locally."""
+ """Total shuffle bytes and records read (includes both data read locally and data read from
+ remote executors). """
+
+ val SHUFFLE_READ_REMOTE_SIZE =
+ """Total shuffle bytes read from remote executors. This is a subset of the shuffle
+ read bytes; the remaining shuffle data is read locally. """
val GETTING_RESULT_TIME =
"""Time that the driver spends fetching task results from workers. If this is large, consider
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 9be65a4a39a09..ec68837a1516c 100644
--- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
@@ -47,7 +47,7 @@ private[spark] abstract class WebUI(
protected val handlers = ArrayBuffer[ServletContextHandler]()
protected var serverInfo: Option[ServerInfo] = None
protected val localHostName = Utils.localHostName()
- protected val publicHostName = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName)
+ protected val publicHostName = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName)
private val className = Utils.getFormattedClassName(this)
def getBasePath: String = basePath
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala
index c82730f524eb7..f0ae95bb8c812 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala
@@ -43,7 +43,7 @@ private[ui] class ExecutorThreadDumpPage(parent: ExecutorsTab) extends WebUIPage
}
id
}.getOrElse {
- return Text(s"Missing executorId parameter")
+ throw new IllegalArgumentException(s"Missing executorId parameter")
}
val time = System.currentTimeMillis()
val maybeThreadDump = sc.get.getExecutorThreadDump(executorId)
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 363cb96de7998..956608d7c0cbe 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
@@ -26,7 +26,8 @@ import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage}
import org.apache.spark.util.Utils
/** Summary information about an executor to display in the UI. */
-private case class ExecutorSummaryInfo(
+// Needs to be private[ui] because of a false positive MiMa failure.
+private[ui] case class ExecutorSummaryInfo(
id: String,
hostPort: String,
rddBlocks: Int,
@@ -40,7 +41,8 @@ private case class ExecutorSummaryInfo(
totalInputBytes: Long,
totalShuffleRead: Long,
totalShuffleWrite: Long,
- maxMemory: Long)
+ maxMemory: Long,
+ executorLogs: Map[String, String])
private[ui] class ExecutorsPage(
parent: ExecutorsTab,
@@ -55,6 +57,7 @@ private[ui] class ExecutorsPage(
val diskUsed = storageStatusList.map(_.diskUsed).sum
val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId)
val execInfoSorted = execInfo.sortBy(_.id)
+ val logsExist = execInfo.filter(_.executorLogs.nonEmpty).nonEmpty
val execTable =
@@ -79,10 +82,11 @@ private[ui] class ExecutorsPage(
Shuffle Write
+ {if (logsExist) Logs else Seq.empty}
{if (threadDumpEnabled) Thread Dump else Seq.empty}
- {execInfoSorted.map(execRow)}
+ {execInfoSorted.map(execRow(_, logsExist))}
@@ -107,7 +111,7 @@ private[ui] class ExecutorsPage(
}
/** Render an HTML row representing an executor */
- private def execRow(info: ExecutorSummaryInfo): Seq[Node] = {
+ private def execRow(info: ExecutorSummaryInfo, logsExist: Boolean): Seq[Node] = {
val maximumMemory = info.maxMemory
val memoryUsed = info.memoryUsed
val diskUsed = info.diskUsed
@@ -138,6 +142,21 @@ private[ui] class ExecutorsPage(
{Utils.bytesToString(info.totalShuffleWrite)}
+ {
+ if (logsExist) {
+
+ {
+ info.executorLogs.map { case (logName, logUrl) =>
+
+
+ {logName}
+
+
+ }
+ }
+
+ }
+ }
{
if (threadDumpEnabled) {
val encodedId = URLEncoder.encode(info.id, "UTF-8")
@@ -168,6 +187,7 @@ private[ui] class ExecutorsPage(
val totalInputBytes = listener.executorToInputBytes.getOrElse(execId, 0L)
val totalShuffleRead = listener.executorToShuffleRead.getOrElse(execId, 0L)
val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0L)
+ val executorLogs = listener.executorToLogUrls.getOrElse(execId, Map.empty)
new ExecutorSummaryInfo(
execId,
@@ -183,7 +203,8 @@ private[ui] class ExecutorsPage(
totalInputBytes,
totalShuffleRead,
totalShuffleWrite,
- maxMem
+ maxMem,
+ executorLogs
)
}
}
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 dd1c2b78c4094..3afd7ef07d7c9 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
@@ -48,12 +48,20 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp
val executorToTasksFailed = HashMap[String, Int]()
val executorToDuration = HashMap[String, Long]()
val executorToInputBytes = HashMap[String, Long]()
+ val executorToInputRecords = HashMap[String, Long]()
val executorToOutputBytes = HashMap[String, Long]()
+ val executorToOutputRecords = HashMap[String, Long]()
val executorToShuffleRead = HashMap[String, Long]()
val executorToShuffleWrite = HashMap[String, Long]()
+ val executorToLogUrls = HashMap[String, Map[String, String]]()
def storageStatusList = storageStatusListener.storageStatusList
+ override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded) = synchronized {
+ val eid = executorAdded.executorId
+ executorToLogUrls(eid) = executorAdded.executorInfo.logUrlMap
+ }
+
override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized {
val eid = taskStart.taskInfo.executorId
executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1
@@ -78,10 +86,14 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp
metrics.inputMetrics.foreach { inputMetrics =>
executorToInputBytes(eid) =
executorToInputBytes.getOrElse(eid, 0L) + inputMetrics.bytesRead
+ executorToInputRecords(eid) =
+ executorToInputRecords.getOrElse(eid, 0L) + inputMetrics.recordsRead
}
metrics.outputMetrics.foreach { outputMetrics =>
executorToOutputBytes(eid) =
executorToOutputBytes.getOrElse(eid, 0L) + outputMetrics.bytesWritten
+ executorToOutputRecords(eid) =
+ executorToOutputRecords.getOrElse(eid, 0L) + outputMetrics.recordsWritten
}
metrics.shuffleReadMetrics.foreach { shuffleRead =>
executorToShuffleRead(eid) =
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
index 045c69da06feb..bd923d78a86ce 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
@@ -42,7 +42,9 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
}
def makeRow(job: JobUIData): Seq[Node] = {
- val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max)
+ val lastStageInfo = Option(job.stageIds)
+ .filter(_.nonEmpty)
+ .flatMap { ids => listener.stageIdToInfo.get(ids.max) }
val lastStageData = lastStageInfo.flatMap { s =>
listener.stageIdToData.get((s.stageId, s.attemptId))
}
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 9836d11a6d85f..1f8536d1b7195 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
@@ -36,6 +36,20 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage
/** Special table which merges two header cells. */
private def executorTable[T](): Seq[Node] = {
+ val stageData = listener.stageIdToData.get((stageId, stageAttemptId))
+ var hasInput = false
+ var hasOutput = false
+ var hasShuffleWrite = false
+ var hasShuffleRead = false
+ var hasBytesSpilled = false
+ stageData.foreach(data => {
+ hasInput = data.hasInput
+ hasOutput = data.hasOutput
+ hasShuffleRead = data.hasShuffleRead
+ hasShuffleWrite = data.hasShuffleWrite
+ hasBytesSpilled = data.hasBytesSpilled
+ })
+
Executor ID
@@ -44,12 +58,32 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage
Total Tasks
Failed Tasks
Succeeded Tasks
- Input
- Output
- Shuffle Read
- Shuffle Write
- Shuffle Spill (Memory)
- Shuffle Spill (Disk)
+ {if (hasInput) {
+
+ Input Size / Records
+
+ }}
+ {if (hasOutput) {
+
+ Output Size / Records
+
+ }}
+ {if (hasShuffleRead) {
+
+
+ Shuffle Read Size / Records
+
+ }}
+ {if (hasShuffleWrite) {
+
+
+ Shuffle Write Size / Records
+
+ }}
+ {if (hasBytesSpilled) {
+ Shuffle Spill (Memory)
+ Shuffle Spill (Disk)
+ }}
{createExecutorTable()}
@@ -76,18 +110,34 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage
{v.failedTasks + v.succeededTasks}
{v.failedTasks}
{v.succeededTasks}
-
- {Utils.bytesToString(v.inputBytes)}
-
- {Utils.bytesToString(v.outputBytes)}
-
- {Utils.bytesToString(v.shuffleRead)}
-
- {Utils.bytesToString(v.shuffleWrite)}
-
- {Utils.bytesToString(v.memoryBytesSpilled)}
-
- {Utils.bytesToString(v.diskBytesSpilled)}
+ {if (stageData.hasInput) {
+
+ {s"${Utils.bytesToString(v.inputBytes)} / ${v.inputRecords}"}
+
+ }}
+ {if (stageData.hasOutput) {
+
+ {s"${Utils.bytesToString(v.outputBytes)} / ${v.outputRecords}"}
+
+ }}
+ {if (stageData.hasShuffleRead) {
+
+ {s"${Utils.bytesToString(v.shuffleRead)} / ${v.shuffleReadRecords}"}
+
+ }}
+ {if (stageData.hasShuffleWrite) {
+
+ {s"${Utils.bytesToString(v.shuffleWrite)} / ${v.shuffleWriteRecords}"}
+
+ }}
+ {if (stageData.hasBytesSpilled) {
+
+ {Utils.bytesToString(v.memoryBytesSpilled)}
+
+
+ {Utils.bytesToString(v.diskBytesSpilled)}
+
+ }}
}
case None =>
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
index 77d36209c6048..7541d3e9c72e7 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
@@ -32,7 +32,10 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
def render(request: HttpServletRequest): Seq[Node] = {
listener.synchronized {
- val jobId = request.getParameter("id").toInt
+ val parameterId = request.getParameter("id")
+ require(parameterId != null && parameterId.nonEmpty, "Missing id parameter")
+
+ val jobId = parameterId.toInt
val jobDataOption = listener.jobIdToData.get(jobId)
if (jobDataOption.isEmpty) {
val content =
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 4d200eeda86b9..937d95a934b59 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
@@ -203,6 +203,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
for (stageId <- jobData.stageIds) {
stageIdToActiveJobIds.get(stageId).foreach { jobsUsingStage =>
jobsUsingStage.remove(jobEnd.jobId)
+ if (jobsUsingStage.isEmpty) {
+ stageIdToActiveJobIds.remove(stageId)
+ }
stageIdToInfo.get(stageId).foreach { stageInfo =>
if (stageInfo.submissionTime.isEmpty) {
// if this stage is pending, it won't complete, so mark it as "skipped":
@@ -394,24 +397,48 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
stageData.shuffleWriteBytes += shuffleWriteDelta
execSummary.shuffleWrite += shuffleWriteDelta
+ val shuffleWriteRecordsDelta =
+ (taskMetrics.shuffleWriteMetrics.map(_.shuffleRecordsWritten).getOrElse(0L)
+ - oldMetrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleRecordsWritten).getOrElse(0L))
+ stageData.shuffleWriteRecords += shuffleWriteRecordsDelta
+ execSummary.shuffleWriteRecords += shuffleWriteRecordsDelta
+
val shuffleReadDelta =
- (taskMetrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L)
- - oldMetrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead).getOrElse(0L))
- stageData.shuffleReadBytes += shuffleReadDelta
+ (taskMetrics.shuffleReadMetrics.map(_.totalBytesRead).getOrElse(0L)
+ - oldMetrics.flatMap(_.shuffleReadMetrics).map(_.totalBytesRead).getOrElse(0L))
+ stageData.shuffleReadTotalBytes += shuffleReadDelta
execSummary.shuffleRead += shuffleReadDelta
+ val shuffleReadRecordsDelta =
+ (taskMetrics.shuffleReadMetrics.map(_.recordsRead).getOrElse(0L)
+ - oldMetrics.flatMap(_.shuffleReadMetrics).map(_.recordsRead).getOrElse(0L))
+ stageData.shuffleReadRecords += shuffleReadRecordsDelta
+ execSummary.shuffleReadRecords += shuffleReadRecordsDelta
+
val inputBytesDelta =
(taskMetrics.inputMetrics.map(_.bytesRead).getOrElse(0L)
- oldMetrics.flatMap(_.inputMetrics).map(_.bytesRead).getOrElse(0L))
stageData.inputBytes += inputBytesDelta
execSummary.inputBytes += inputBytesDelta
+ val inputRecordsDelta =
+ (taskMetrics.inputMetrics.map(_.recordsRead).getOrElse(0L)
+ - oldMetrics.flatMap(_.inputMetrics).map(_.recordsRead).getOrElse(0L))
+ stageData.inputRecords += inputRecordsDelta
+ execSummary.inputRecords += inputRecordsDelta
+
val outputBytesDelta =
(taskMetrics.outputMetrics.map(_.bytesWritten).getOrElse(0L)
- oldMetrics.flatMap(_.outputMetrics).map(_.bytesWritten).getOrElse(0L))
stageData.outputBytes += outputBytesDelta
execSummary.outputBytes += outputBytesDelta
+ val outputRecordsDelta =
+ (taskMetrics.outputMetrics.map(_.recordsWritten).getOrElse(0L)
+ - oldMetrics.flatMap(_.outputMetrics).map(_.recordsWritten).getOrElse(0L))
+ stageData.outputRecords += outputRecordsDelta
+ execSummary.outputRecords += outputRecordsDelta
+
val diskSpillDelta =
taskMetrics.diskBytesSpilled - oldMetrics.map(_.diskBytesSpilled).getOrElse(0L)
stageData.diskBytesSpilled += diskSpillDelta
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
index 5fc6cc7533150..f47cdc935e539 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
@@ -32,6 +32,8 @@ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") {
def render(request: HttpServletRequest): Seq[Node] = {
listener.synchronized {
val poolName = request.getParameter("poolname")
+ require(poolName != null && poolName.nonEmpty, "Missing poolname parameter")
+
val poolToActiveStages = listener.poolToActiveStages
val activeStages = poolToActiveStages.get(poolName) match {
case Some(s) => s.values.toSeq
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 d8be1b20b3acd..110f8780a9a12 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
@@ -36,8 +36,14 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
def render(request: HttpServletRequest): Seq[Node] = {
listener.synchronized {
- val stageId = request.getParameter("id").toInt
- val stageAttemptId = request.getParameter("attempt").toInt
+ val parameterId = request.getParameter("id")
+ require(parameterId != null && parameterId.nonEmpty, "Missing id parameter")
+
+ val parameterAttempt = request.getParameter("attempt")
+ require(parameterAttempt != null && parameterAttempt.nonEmpty, "Missing attempt parameter")
+
+ val stageId = parameterId.toInt
+ val stageAttemptId = parameterAttempt.toInt
val stageDataOption = listener.stageIdToData.get((stageId, stageAttemptId))
if (stageDataOption.isEmpty || stageDataOption.get.taskData.isEmpty) {
@@ -56,11 +62,6 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
val numCompleted = tasks.count(_.taskInfo.finished)
val accumulables = listener.stageIdToData((stageId, stageAttemptId)).accumulables
val hasAccumulators = accumulables.size > 0
- val hasInput = stageData.inputBytes > 0
- val hasOutput = stageData.outputBytes > 0
- val hasShuffleRead = stageData.shuffleReadBytes > 0
- val hasShuffleWrite = stageData.shuffleWriteBytes > 0
- val hasBytesSpilled = stageData.memoryBytesSpilled > 0 && stageData.diskBytesSpilled > 0
val summary =
@@ -69,31 +70,33 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
Total task time across all tasks:
{UIUtils.formatDuration(stageData.executorRunTime)}
- {if (hasInput) {
+ {if (stageData.hasInput) {
-
- Input:
- {Utils.bytesToString(stageData.inputBytes)}
+ Input Size / Records:
+ {s"${Utils.bytesToString(stageData.inputBytes)} / ${stageData.inputRecords}"}
}}
- {if (hasOutput) {
+ {if (stageData.hasOutput) {
-
Output:
- {Utils.bytesToString(stageData.outputBytes)}
+ {s"${Utils.bytesToString(stageData.outputBytes)} / ${stageData.outputRecords}"}
}}
- {if (hasShuffleRead) {
+ {if (stageData.hasShuffleRead) {
-
Shuffle read:
- {Utils.bytesToString(stageData.shuffleReadBytes)}
+ {s"${Utils.bytesToString(stageData.shuffleReadTotalBytes)} / " +
+ s"${stageData.shuffleReadRecords}"}
}}
- {if (hasShuffleWrite) {
+ {if (stageData.hasShuffleWrite) {
-
Shuffle write:
- {Utils.bytesToString(stageData.shuffleWriteBytes)}
+ {s"${Utils.bytesToString(stageData.shuffleWriteBytes)} / " +
+ s"${stageData.shuffleWriteRecords}"}
}}
- {if (hasBytesSpilled) {
+ {if (stageData.hasBytesSpilled) {
-
Shuffle spill (memory):
{Utils.bytesToString(stageData.memoryBytesSpilled)}
@@ -132,7 +135,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
Task Deserialization Time
- {if (hasShuffleRead) {
+ {if (stageData.hasShuffleRead) {
-
@@ -140,6 +143,13 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
Shuffle Read Blocked Time
+ -
+
+
+ Shuffle Remote Reads
+
+
}}
-
+ getDistributionQuantiles(times).map { millis =>
{UIUtils.formatDuration(millis.toLong)}
}
}
@@ -273,35 +294,86 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
getFormattedTimeQuantiles(schedulerDelays)
def getFormattedSizeQuantiles(data: Seq[Double]) =
- Distribution(data).get.getQuantiles().map(d => {Utils.bytesToString(d.toLong)} )
+ getDistributionQuantiles(data).map(d => {Utils.bytesToString(d.toLong)} )
+
+ def getFormattedSizeQuantilesWithRecords(data: Seq[Double], records: Seq[Double]) = {
+ val recordDist = getDistributionQuantiles(records).iterator
+ getDistributionQuantiles(data).map(d =>
+ {s"${Utils.bytesToString(d.toLong)} / ${recordDist.next().toLong}"}
+ )
+ }
val inputSizes = validTasks.map { case TaskUIData(_, metrics, _) =>
metrics.get.inputMetrics.map(_.bytesRead).getOrElse(0L).toDouble
}
- val inputQuantiles = Input +: getFormattedSizeQuantiles(inputSizes)
+
+ val inputRecords = validTasks.map { case TaskUIData(_, metrics, _) =>
+ metrics.get.inputMetrics.map(_.recordsRead).getOrElse(0L).toDouble
+ }
+
+ val inputQuantiles = Input Size / Records +:
+ getFormattedSizeQuantilesWithRecords(inputSizes, inputRecords)
val outputSizes = validTasks.map { case TaskUIData(_, metrics, _) =>
metrics.get.outputMetrics.map(_.bytesWritten).getOrElse(0L).toDouble
}
- val outputQuantiles = Output +: getFormattedSizeQuantiles(outputSizes)
+
+ val outputRecords = validTasks.map { case TaskUIData(_, metrics, _) =>
+ metrics.get.outputMetrics.map(_.recordsWritten).getOrElse(0L).toDouble
+ }
+
+ val outputQuantiles = Output Size / Records +:
+ getFormattedSizeQuantilesWithRecords(outputSizes, outputRecords)
val shuffleReadBlockedTimes = validTasks.map { case TaskUIData(_, metrics, _) =>
metrics.get.shuffleReadMetrics.map(_.fetchWaitTime).getOrElse(0L).toDouble
}
- val shuffleReadBlockedQuantiles = Shuffle Read Blocked Time +:
+ val shuffleReadBlockedQuantiles =
+
+
+ Shuffle Read Blocked Time
+
+ +:
getFormattedTimeQuantiles(shuffleReadBlockedTimes)
- val shuffleReadSizes = validTasks.map { case TaskUIData(_, metrics, _) =>
+ val shuffleReadTotalSizes = validTasks.map { case TaskUIData(_, metrics, _) =>
+ metrics.get.shuffleReadMetrics.map(_.totalBytesRead).getOrElse(0L).toDouble
+ }
+ val shuffleReadTotalRecords = validTasks.map { case TaskUIData(_, metrics, _) =>
+ metrics.get.shuffleReadMetrics.map(_.recordsRead).getOrElse(0L).toDouble
+ }
+ val shuffleReadTotalQuantiles =
+
+
+ Shuffle Read Size / Records
+
+ +:
+ getFormattedSizeQuantilesWithRecords(shuffleReadTotalSizes, shuffleReadTotalRecords)
+
+ val shuffleReadRemoteSizes = validTasks.map { case TaskUIData(_, metrics, _) =>
metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble
}
- val shuffleReadQuantiles = Shuffle Read (Remote) +:
- getFormattedSizeQuantiles(shuffleReadSizes)
+ val shuffleReadRemoteQuantiles =
+
+
+ Shuffle Remote Reads
+
+ +:
+ getFormattedSizeQuantiles(shuffleReadRemoteSizes)
val shuffleWriteSizes = validTasks.map { case TaskUIData(_, metrics, _) =>
metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble
}
- val shuffleWriteQuantiles = Shuffle Write +:
- getFormattedSizeQuantiles(shuffleWriteSizes)
+
+ val shuffleWriteRecords = validTasks.map { case TaskUIData(_, metrics, _) =>
+ metrics.get.shuffleWriteMetrics.map(_.shuffleRecordsWritten).getOrElse(0L).toDouble
+ }
+
+ val shuffleWriteQuantiles = Shuffle Write Size / Records +:
+ getFormattedSizeQuantilesWithRecords(shuffleWriteSizes, shuffleWriteRecords)
val memoryBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) =>
metrics.get.memoryBytesSpilled.toDouble
@@ -326,19 +398,22 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
{serializationQuantiles}
,
{gettingResultQuantiles} ,
- if (hasInput) {inputQuantiles} else Nil,
- if (hasOutput) {outputQuantiles} else Nil,
- if (hasShuffleRead) {
+ if (stageData.hasInput) {inputQuantiles} else Nil,
+ if (stageData.hasOutput) {outputQuantiles} else Nil,
+ if (stageData.hasShuffleRead) {
{shuffleReadBlockedQuantiles}
- {shuffleReadQuantiles}
+ {shuffleReadTotalQuantiles}
+
+ {shuffleReadRemoteQuantiles}
+
} else {
Nil
},
- if (hasShuffleWrite) {shuffleWriteQuantiles} else Nil,
- if (hasBytesSpilled) {memoryBytesSpilledQuantiles} else Nil,
- if (hasBytesSpilled) {diskBytesSpilledQuantiles} else Nil)
+ if (stageData.hasShuffleWrite) {shuffleWriteQuantiles} else Nil,
+ if (stageData.hasBytesSpilled) {memoryBytesSpilledQuantiles} else Nil,
+ if (stageData.hasBytesSpilled) {diskBytesSpilledQuantiles} else Nil)
val quantileHeaders = Seq("Metric", "Min", "25th percentile",
"Median", "75th percentile", "Max")
@@ -397,26 +472,36 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
val inputReadable = maybeInput
.map(m => s"${Utils.bytesToString(m.bytesRead)} (${m.readMethod.toString.toLowerCase()})")
.getOrElse("")
+ val inputRecords = maybeInput.map(_.recordsRead.toString).getOrElse("")
val maybeOutput = metrics.flatMap(_.outputMetrics)
val outputSortable = maybeOutput.map(_.bytesWritten.toString).getOrElse("")
val outputReadable = maybeOutput
.map(m => s"${Utils.bytesToString(m.bytesWritten)}")
.getOrElse("")
+ val outputRecords = maybeOutput.map(_.recordsWritten.toString).getOrElse("")
- val maybeShuffleReadBlockedTime = metrics.flatMap(_.shuffleReadMetrics).map(_.fetchWaitTime)
- val shuffleReadBlockedTimeSortable = maybeShuffleReadBlockedTime.map(_.toString).getOrElse("")
+ val maybeShuffleRead = metrics.flatMap(_.shuffleReadMetrics)
+ val shuffleReadBlockedTimeSortable = maybeShuffleRead
+ .map(_.fetchWaitTime.toString).getOrElse("")
val shuffleReadBlockedTimeReadable =
- maybeShuffleReadBlockedTime.map(ms => UIUtils.formatDuration(ms)).getOrElse("")
+ maybeShuffleRead.map(ms => UIUtils.formatDuration(ms.fetchWaitTime)).getOrElse("")
+
+ val totalShuffleBytes = maybeShuffleRead.map(_.totalBytesRead)
+ val shuffleReadSortable = totalShuffleBytes.map(_.toString).getOrElse("")
+ val shuffleReadReadable = totalShuffleBytes.map(Utils.bytesToString).getOrElse("")
+ val shuffleReadRecords = maybeShuffleRead.map(_.recordsRead.toString).getOrElse("")
- val maybeShuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead)
- val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("")
- val shuffleReadReadable = maybeShuffleRead.map(Utils.bytesToString).getOrElse("")
+ val remoteShuffleBytes = maybeShuffleRead.map(_.remoteBytesRead)
+ val shuffleReadRemoteSortable = remoteShuffleBytes.map(_.toString).getOrElse("")
+ val shuffleReadRemoteReadable = remoteShuffleBytes.map(Utils.bytesToString).getOrElse("")
- val maybeShuffleWrite =
- metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten)
- val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("")
- val shuffleWriteReadable = maybeShuffleWrite.map(Utils.bytesToString).getOrElse("")
+ val maybeShuffleWrite = metrics.flatMap(_.shuffleWriteMetrics)
+ val shuffleWriteSortable = maybeShuffleWrite.map(_.shuffleBytesWritten.toString).getOrElse("")
+ val shuffleWriteReadable = maybeShuffleWrite
+ .map(m => s"${Utils.bytesToString(m.shuffleBytesWritten)}").getOrElse("")
+ val shuffleWriteRecords = maybeShuffleWrite
+ .map(_.shuffleRecordsWritten.toString).getOrElse("")
val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime)
val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("")
@@ -472,12 +557,12 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
}}
{if (hasInput) {
- {inputReadable}
+ {s"$inputReadable / $inputRecords"}
}}
{if (hasOutput) {
- {outputReadable}
+ {s"$outputReadable / $outputRecords"}
}}
{if (hasShuffleRead) {
@@ -486,7 +571,11 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
{shuffleReadBlockedTimeReadable}
- {shuffleReadReadable}
+ {s"$shuffleReadReadable / $shuffleReadRecords"}
+
+
+ {shuffleReadRemoteReadable}
}}
{if (hasShuffleWrite) {
@@ -494,7 +583,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
{writeTimeReadable}
- {shuffleWriteReadable}
+ {s"$shuffleWriteReadable / $shuffleWriteRecords"}
}}
{if (hasBytesSpilled) {
@@ -537,15 +626,16 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
}
private def getSchedulerDelay(info: TaskInfo, metrics: TaskMetrics): Long = {
- val totalExecutionTime = {
- if (info.gettingResultTime > 0) {
- (info.gettingResultTime - info.launchTime)
+ val totalExecutionTime =
+ if (info.gettingResult) {
+ info.gettingResultTime - info.launchTime
+ } else if (info.finished) {
+ info.finishTime - info.launchTime
} else {
- (info.finishTime - info.launchTime)
+ 0
}
- }
val executorOverhead = (metrics.executorDeserializeTime +
metrics.resultSerializationTime)
- totalExecutionTime - metrics.executorRunTime - executorOverhead
+ math.max(0, totalExecutionTime - metrics.executorRunTime - executorOverhead)
}
}
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 703d43f9c640d..5865850fa09b5 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
@@ -138,7 +138,7 @@ private[ui] class StageTableBase(
val inputReadWithUnit = if (inputRead > 0) Utils.bytesToString(inputRead) else ""
val outputWrite = stageData.outputBytes
val outputWriteWithUnit = if (outputWrite > 0) Utils.bytesToString(outputWrite) else ""
- val shuffleRead = stageData.shuffleReadBytes
+ val shuffleRead = stageData.shuffleReadTotalBytes
val shuffleReadWithUnit = if (shuffleRead > 0) Utils.bytesToString(shuffleRead) else ""
val shuffleWrite = stageData.shuffleWriteBytes
val shuffleWriteWithUnit = if (shuffleWrite > 0) Utils.bytesToString(shuffleWrite) else ""
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala
index 37cf2c207ba40..9bf67db8acde1 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala
@@ -28,6 +28,7 @@ private[spark] object TaskDetailsClassNames {
val SCHEDULER_DELAY = "scheduler_delay"
val TASK_DESERIALIZATION_TIME = "deserialization_time"
val SHUFFLE_READ_BLOCKED_TIME = "fetch_wait_time"
+ val SHUFFLE_READ_REMOTE_SIZE = "shuffle_read_remote"
val RESULT_SERIALIZATION_TIME = "serialization_time"
val GETTING_RESULT_TIME = "getting_result_time"
}
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
index 01f7e23212c3d..dbf1ceeda1878 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
@@ -31,9 +31,13 @@ private[jobs] object UIData {
var failedTasks : Int = 0
var succeededTasks : Int = 0
var inputBytes : Long = 0
+ var inputRecords : Long = 0
var outputBytes : Long = 0
+ var outputRecords : Long = 0
var shuffleRead : Long = 0
+ var shuffleReadRecords : Long = 0
var shuffleWrite : Long = 0
+ var shuffleWriteRecords : Long = 0
var memoryBytesSpilled : Long = 0
var diskBytesSpilled : Long = 0
}
@@ -73,9 +77,13 @@ private[jobs] object UIData {
var executorRunTime: Long = _
var inputBytes: Long = _
+ var inputRecords: Long = _
var outputBytes: Long = _
- var shuffleReadBytes: Long = _
+ var outputRecords: Long = _
+ var shuffleReadTotalBytes: Long = _
+ var shuffleReadRecords : Long = _
var shuffleWriteBytes: Long = _
+ var shuffleWriteRecords: Long = _
var memoryBytesSpilled: Long = _
var diskBytesSpilled: Long = _
@@ -85,6 +93,12 @@ private[jobs] object UIData {
var accumulables = new HashMap[Long, AccumulableInfo]
var taskData = new HashMap[Long, TaskUIData]
var executorSummary = new HashMap[String, ExecutorSummary]
+
+ def hasInput = inputBytes > 0
+ def hasOutput = outputBytes > 0
+ def hasShuffleRead = shuffleReadTotalBytes > 0
+ def hasShuffleWrite = shuffleWriteBytes > 0
+ def hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0
}
/**
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
index 12d23a92878cf..199f731b92bcc 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
@@ -30,7 +30,10 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
private val listener = parent.listener
def render(request: HttpServletRequest): Seq[Node] = {
- val rddId = request.getParameter("id").toInt
+ val parameterId = request.getParameter("id")
+ require(parameterId != null && parameterId.nonEmpty, "Missing id parameter")
+
+ val rddId = parameterId.toInt
val storageStatusList = listener.storageStatusList
val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse {
// Rather than crashing, render an "RDD Not Found" page
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 4c9b1e3c46f0f..48a6ede05e17b 100644
--- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
@@ -20,6 +20,7 @@ package org.apache.spark.util
import scala.collection.JavaConversions.mapAsJavaMap
import scala.concurrent.Await
import scala.concurrent.duration.{Duration, FiniteDuration}
+import scala.util.Try
import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem}
import akka.pattern.ask
@@ -78,8 +79,6 @@ private[spark] object AkkaUtils extends Logging {
val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off"
val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 6000)
- val akkaFailureDetector =
- conf.getDouble("spark.akka.failure-detector.threshold", 300.0)
val akkaHeartBeatInterval = conf.getInt("spark.akka.heartbeat.interval", 1000)
val secretKey = securityManager.getSecretKey()
@@ -91,8 +90,11 @@ private[spark] object AkkaUtils extends Logging {
val secureCookie = if (isAuthOn) secretKey else ""
logDebug(s"In createActorSystem, requireCookie is: $requireCookie")
- val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]).withFallback(
- ConfigFactory.parseString(
+ val akkaSslConfig = securityManager.akkaSSLOptions.createAkkaConfig
+ .getOrElse(ConfigFactory.empty())
+
+ val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String])
+ .withFallback(akkaSslConfig).withFallback(ConfigFactory.parseString(
s"""
|akka.daemonic = on
|akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
@@ -102,7 +104,6 @@ private[spark] object AkkaUtils extends Logging {
|akka.remote.secure-cookie = "$secureCookie"
|akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s
|akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s
- |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector
|akka.actor.provider = "akka.remote.RemoteActorRefProvider"
|akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport"
|akka.remote.netty.tcp.hostname = "$host"
@@ -214,7 +215,7 @@ private[spark] object AkkaUtils extends Logging {
val driverHost: String = conf.get("spark.driver.host", "localhost")
val driverPort: Int = conf.getInt("spark.driver.port", 7077)
Utils.checkHost(driverHost, "Expected hostname")
- val url = s"akka.tcp://$driverActorSystemName@$driverHost:$driverPort/user/$name"
+ val url = address(protocol(actorSystem), driverActorSystemName, driverHost, driverPort, name)
val timeout = AkkaUtils.lookupTimeout(conf)
logInfo(s"Connecting to $name: $url")
Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout)
@@ -228,9 +229,33 @@ private[spark] object AkkaUtils extends Logging {
actorSystem: ActorSystem): ActorRef = {
val executorActorSystemName = SparkEnv.executorActorSystemName
Utils.checkHost(host, "Expected hostname")
- val url = s"akka.tcp://$executorActorSystemName@$host:$port/user/$name"
+ val url = address(protocol(actorSystem), executorActorSystemName, host, port, name)
val timeout = AkkaUtils.lookupTimeout(conf)
logInfo(s"Connecting to $name: $url")
Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout)
}
+
+ def protocol(actorSystem: ActorSystem): String = {
+ val akkaConf = actorSystem.settings.config
+ val sslProp = "akka.remote.netty.tcp.enable-ssl"
+ protocol(akkaConf.hasPath(sslProp) && akkaConf.getBoolean(sslProp))
+ }
+
+ def protocol(ssl: Boolean = false): String = {
+ if (ssl) {
+ "akka.ssl.tcp"
+ } else {
+ "akka.tcp"
+ }
+ }
+
+ def address(
+ protocol: String,
+ systemName: String,
+ host: String,
+ port: Any,
+ actorName: String): String = {
+ s"$protocol://$systemName@$host:$port/user/$actorName"
+ }
+
}
diff --git a/core/src/main/scala/org/apache/spark/util/Clock.scala b/core/src/main/scala/org/apache/spark/util/Clock.scala
index 97c2b45aabf28..e92ed11bd165b 100644
--- a/core/src/main/scala/org/apache/spark/util/Clock.scala
+++ b/core/src/main/scala/org/apache/spark/util/Clock.scala
@@ -21,9 +21,47 @@ package org.apache.spark.util
* An interface to represent clocks, so that they can be mocked out in unit tests.
*/
private[spark] trait Clock {
- def getTime(): Long
+ def getTimeMillis(): Long
+ def waitTillTime(targetTime: Long): Long
}
-private[spark] object SystemClock extends Clock {
- def getTime(): Long = System.currentTimeMillis()
+/**
+ * A clock backed by the actual time from the OS as reported by the `System` API.
+ */
+private[spark] class SystemClock extends Clock {
+
+ val minPollTime = 25L
+
+ /**
+ * @return the same time (milliseconds since the epoch)
+ * as is reported by `System.currentTimeMillis()`
+ */
+ def getTimeMillis(): Long = System.currentTimeMillis()
+
+ /**
+ * @param targetTime block until the current time is at least this value
+ * @return current system time when wait has completed
+ */
+ def waitTillTime(targetTime: Long): Long = {
+ var currentTime = 0L
+ currentTime = System.currentTimeMillis()
+
+ var waitTime = targetTime - currentTime
+ if (waitTime <= 0) {
+ return currentTime
+ }
+
+ val pollTime = math.max(waitTime / 10.0, minPollTime).toLong
+
+ while (true) {
+ currentTime = System.currentTimeMillis()
+ waitTime = targetTime - currentTime
+ if (waitTime <= 0) {
+ return currentTime
+ }
+ val sleepTime = math.min(waitTime, pollTime)
+ Thread.sleep(sleepTime)
+ }
+ -1
+ }
}
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 414bc49a57f8a..474f79fb756f6 100644
--- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
@@ -32,7 +32,6 @@ import org.apache.spark.executor._
import org.apache.spark.scheduler._
import org.apache.spark.storage._
import org.apache.spark._
-import org.apache.hadoop.hdfs.web.JsonUtil
/**
* Serializes SparkListener events to/from JSON. This protocol provides strong backwards-
@@ -90,6 +89,8 @@ private[spark] object JsonProtocol {
executorAddedToJson(executorAdded)
case executorRemoved: SparkListenerExecutorRemoved =>
executorRemovedToJson(executorRemoved)
+ case logStart: SparkListenerLogStart =>
+ logStartToJson(logStart)
// These aren't used, but keeps compiler happy
case SparkListenerExecutorMetricsUpdate(_, _) => JNothing
}
@@ -215,6 +216,11 @@ private[spark] object JsonProtocol {
("Removed Reason" -> executorRemoved.reason)
}
+ def logStartToJson(logStart: SparkListenerLogStart): JValue = {
+ ("Event" -> Utils.getFormattedClassName(logStart)) ~
+ ("Spark Version" -> SPARK_VERSION)
+ }
+
/** ------------------------------------------------------------------- *
* JSON serialization methods for classes SparkListenerEvents depend on |
* -------------------------------------------------------------------- */
@@ -294,22 +300,27 @@ private[spark] object JsonProtocol {
("Remote Blocks Fetched" -> shuffleReadMetrics.remoteBlocksFetched) ~
("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~
("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~
- ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead)
+ ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead) ~
+ ("Local Bytes Read" -> shuffleReadMetrics.localBytesRead) ~
+ ("Total Records Read" -> shuffleReadMetrics.recordsRead)
}
def shuffleWriteMetricsToJson(shuffleWriteMetrics: ShuffleWriteMetrics): JValue = {
("Shuffle Bytes Written" -> shuffleWriteMetrics.shuffleBytesWritten) ~
- ("Shuffle Write Time" -> shuffleWriteMetrics.shuffleWriteTime)
+ ("Shuffle Write Time" -> shuffleWriteMetrics.shuffleWriteTime) ~
+ ("Shuffle Records Written" -> shuffleWriteMetrics.shuffleRecordsWritten)
}
def inputMetricsToJson(inputMetrics: InputMetrics): JValue = {
("Data Read Method" -> inputMetrics.readMethod.toString) ~
- ("Bytes Read" -> inputMetrics.bytesRead)
+ ("Bytes Read" -> inputMetrics.bytesRead) ~
+ ("Records Read" -> inputMetrics.recordsRead)
}
def outputMetricsToJson(outputMetrics: OutputMetrics): JValue = {
("Data Write Method" -> outputMetrics.writeMethod.toString) ~
- ("Bytes Written" -> outputMetrics.bytesWritten)
+ ("Bytes Written" -> outputMetrics.bytesWritten) ~
+ ("Records Written" -> outputMetrics.recordsWritten)
}
def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = {
@@ -384,7 +395,8 @@ private[spark] object JsonProtocol {
def executorInfoToJson(executorInfo: ExecutorInfo): JValue = {
("Host" -> executorInfo.executorHost) ~
- ("Total Cores" -> executorInfo.totalCores)
+ ("Total Cores" -> executorInfo.totalCores) ~
+ ("Log Urls" -> mapToJson(executorInfo.logUrlMap))
}
/** ------------------------------ *
@@ -442,6 +454,7 @@ private[spark] object JsonProtocol {
val applicationEnd = Utils.getFormattedClassName(SparkListenerApplicationEnd)
val executorAdded = Utils.getFormattedClassName(SparkListenerExecutorAdded)
val executorRemoved = Utils.getFormattedClassName(SparkListenerExecutorRemoved)
+ val logStart = Utils.getFormattedClassName(SparkListenerLogStart)
(json \ "Event").extract[String] match {
case `stageSubmitted` => stageSubmittedFromJson(json)
@@ -459,6 +472,7 @@ private[spark] object JsonProtocol {
case `applicationEnd` => applicationEndFromJson(json)
case `executorAdded` => executorAddedFromJson(json)
case `executorRemoved` => executorRemovedFromJson(json)
+ case `logStart` => logStartFromJson(json)
}
}
@@ -569,6 +583,11 @@ private[spark] object JsonProtocol {
SparkListenerExecutorRemoved(time, executorId, reason)
}
+ def logStartFromJson(json: JValue): SparkListenerLogStart = {
+ val sparkVersion = (json \ "Spark Version").extract[String]
+ SparkListenerLogStart(sparkVersion)
+ }
+
/** --------------------------------------------------------------------- *
* JSON deserialization methods for classes SparkListenerEvents depend on |
* ---------------------------------------------------------------------- */
@@ -670,6 +689,8 @@ private[spark] object JsonProtocol {
metrics.incLocalBlocksFetched((json \ "Local Blocks Fetched").extract[Int])
metrics.incFetchWaitTime((json \ "Fetch Wait Time").extract[Long])
metrics.incRemoteBytesRead((json \ "Remote Bytes Read").extract[Long])
+ metrics.incLocalBytesRead((json \ "Local Bytes Read").extractOpt[Long].getOrElse(0))
+ metrics.incRecordsRead((json \ "Total Records Read").extractOpt[Long].getOrElse(0))
metrics
}
@@ -677,13 +698,16 @@ private[spark] object JsonProtocol {
val metrics = new ShuffleWriteMetrics
metrics.incShuffleBytesWritten((json \ "Shuffle Bytes Written").extract[Long])
metrics.incShuffleWriteTime((json \ "Shuffle Write Time").extract[Long])
+ metrics.setShuffleRecordsWritten((json \ "Shuffle Records Written")
+ .extractOpt[Long].getOrElse(0))
metrics
}
def inputMetricsFromJson(json: JValue): InputMetrics = {
val metrics = new InputMetrics(
DataReadMethod.withName((json \ "Data Read Method").extract[String]))
- metrics.addBytesRead((json \ "Bytes Read").extract[Long])
+ metrics.incBytesRead((json \ "Bytes Read").extract[Long])
+ metrics.incRecordsRead((json \ "Records Read").extractOpt[Long].getOrElse(0))
metrics
}
@@ -691,6 +715,7 @@ private[spark] object JsonProtocol {
val metrics = new OutputMetrics(
DataWriteMethod.withName((json \ "Data Write Method").extract[String]))
metrics.setBytesWritten((json \ "Bytes Written").extract[Long])
+ metrics.setRecordsWritten((json \ "Records Written").extractOpt[Long].getOrElse(0))
metrics
}
@@ -793,7 +818,8 @@ private[spark] object JsonProtocol {
def executorInfoFromJson(json: JValue): ExecutorInfo = {
val executorHost = (json \ "Host").extract[String]
val totalCores = (json \ "Total Cores").extract[Int]
- new ExecutorInfo(executorHost, totalCores)
+ val logUrls = mapFromJson(json \ "Log Urls").toMap
+ new ExecutorInfo(executorHost, totalCores, logUrls)
}
/** -------------------------------- *
diff --git a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala
index bd0aa4dc4650f..d60b8b9a31a9b 100644
--- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala
@@ -28,7 +28,8 @@ import org.apache.spark.Logging
*/
private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging {
- private val listeners = new CopyOnWriteArrayList[L]
+ // Marked `private[spark]` for access in tests.
+ private[spark] val listeners = new CopyOnWriteArrayList[L]
/**
* Add a listener to listen events. This method is thread-safe and can be called in any thread.
diff --git a/core/src/main/scala/org/apache/spark/util/ManualClock.scala b/core/src/main/scala/org/apache/spark/util/ManualClock.scala
new file mode 100644
index 0000000000000..cf89c1782fd67
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/ManualClock.scala
@@ -0,0 +1,69 @@
+/*
+ * 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
+
+/**
+ * A `Clock` whose time can be manually set and modified. Its reported time does not change
+ * as time elapses, but only as its time is modified by callers. This is mainly useful for
+ * testing.
+ *
+ * @param time initial time (in milliseconds since the epoch)
+ */
+private[spark] class ManualClock(private var time: Long) extends Clock {
+
+ /**
+ * @return `ManualClock` with initial time 0
+ */
+ def this() = this(0L)
+
+ def getTimeMillis(): Long =
+ synchronized {
+ time
+ }
+
+ /**
+ * @param timeToSet new time (in milliseconds) that the clock should represent
+ */
+ def setTime(timeToSet: Long) =
+ synchronized {
+ time = timeToSet
+ notifyAll()
+ }
+
+ /**
+ * @param timeToAdd time (in milliseconds) to add to the clock's time
+ */
+ def advance(timeToAdd: Long) =
+ synchronized {
+ time += timeToAdd
+ notifyAll()
+ }
+
+ /**
+ * @param targetTime block until the clock time is set or advanced to at least this time
+ * @return current time reported by the clock when waiting finishes
+ */
+ def waitTillTime(targetTime: Long): Long =
+ synchronized {
+ while (time < targetTime) {
+ wait(100)
+ }
+ getTimeMillis()
+ }
+
+}
diff --git a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala
new file mode 100644
index 0000000000000..1e0ba5c28754a
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.net.{URLClassLoader, URL}
+import java.util.Enumeration
+import java.util.concurrent.ConcurrentHashMap
+
+import scala.collection.JavaConversions._
+
+/**
+ * URL class loader that exposes the `addURL` and `getURLs` methods in URLClassLoader.
+ */
+private[spark] class MutableURLClassLoader(urls: Array[URL], parent: ClassLoader)
+ extends URLClassLoader(urls, parent) {
+
+ override def addURL(url: URL): Unit = {
+ super.addURL(url)
+ }
+
+ override def getURLs(): Array[URL] = {
+ super.getURLs()
+ }
+
+}
+
+/**
+ * A mutable class loader that gives preference to its own URLs over the parent class loader
+ * when loading classes and resources.
+ */
+private[spark] class ChildFirstURLClassLoader(urls: Array[URL], parent: ClassLoader)
+ extends MutableURLClassLoader(urls, null) {
+
+ private val parentClassLoader = new ParentClassLoader(parent)
+
+ /**
+ * Used to implement fine-grained class loading locks similar to what is done by Java 7. This
+ * prevents deadlock issues when using non-hierarchical class loaders.
+ *
+ * Note that due to Java 6 compatibility (and some issues with implementing class loaders in
+ * Scala), Java 7's `ClassLoader.registerAsParallelCapable` method is not called.
+ */
+ private val locks = new ConcurrentHashMap[String, Object]()
+
+ override def loadClass(name: String, resolve: Boolean): Class[_] = {
+ var lock = locks.get(name)
+ if (lock == null) {
+ val newLock = new Object()
+ lock = locks.putIfAbsent(name, newLock)
+ if (lock == null) {
+ lock = newLock
+ }
+ }
+
+ lock.synchronized {
+ try {
+ super.loadClass(name, resolve)
+ } catch {
+ case e: ClassNotFoundException =>
+ parentClassLoader.loadClass(name, resolve)
+ }
+ }
+ }
+
+ override def getResource(name: String): URL = {
+ val url = super.findResource(name)
+ val res = if (url != null) url else parentClassLoader.getResource(name)
+ res
+ }
+
+ override def getResources(name: String): Enumeration[URL] = {
+ val urls = super.findResources(name)
+ val res =
+ if (urls != null && urls.hasMoreElements()) {
+ urls
+ } else {
+ parentClassLoader.getResources(name)
+ }
+ res
+ }
+
+ override def addURL(url: URL) {
+ super.addURL(url)
+ }
+
+}
diff --git a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala
index 3abc12681fe9a..6d8d9e8da3678 100644
--- a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala
+++ b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala
@@ -18,7 +18,7 @@
package org.apache.spark.util
/**
- * A class loader which makes findClass accesible to the child
+ * A class loader which makes some protected methods in ClassLoader accesible.
*/
private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) {
@@ -29,4 +29,9 @@ private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(
override def loadClass(name: String): Class[_] = {
super.loadClass(name)
}
+
+ override def loadClass(name: String, resolve: Boolean): Class[_] = {
+ super.loadClass(name, resolve)
+ }
+
}
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 86ac307fc84ba..d3dc1d09cb7b4 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -21,8 +21,9 @@ import java.io._
import java.lang.management.ManagementFactory
import java.net._
import java.nio.ByteBuffer
-import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor}
-import java.util.{Locale, Properties, Random, UUID}
+import java.util.{Properties, Locale, Random, UUID}
+import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor}
+import javax.net.ssl.HttpsURLConnection
import scala.collection.JavaConversions._
import scala.collection.Map
@@ -37,6 +38,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder
import org.apache.commons.lang3.SystemUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, FileUtil, Path}
+import org.apache.hadoop.security.UserGroupInformation
import org.apache.log4j.PropertyConfigurator
import org.eclipse.jetty.util.MultiException
import org.json4s._
@@ -60,6 +62,9 @@ private[spark] object CallSite {
private[spark] object Utils extends Logging {
val random = new Random()
+ private val MAX_DIR_CREATION_ATTEMPTS: Int = 10
+ @volatile private var localRootDirs: Array[String] = null
+
/** Serialize an object using Java serialization */
def serialize[T](o: T): Array[Byte] = {
val bos = new ByteArrayOutputStream()
@@ -209,8 +214,8 @@ private[spark] object Utils extends Logging {
// Is the path already registered to be deleted via a shutdown hook ?
def hasShutdownDeleteTachyonDir(file: TachyonFile): Boolean = {
val absolutePath = file.getPath()
- shutdownDeletePaths.synchronized {
- shutdownDeletePaths.contains(absolutePath)
+ shutdownDeleteTachyonPaths.synchronized {
+ shutdownDeleteTachyonPaths.contains(absolutePath)
}
}
@@ -246,13 +251,28 @@ private[spark] object Utils extends Logging {
retval
}
+ /**
+ * JDK equivalent of `chmod 700 file`.
+ *
+ * @param file the file whose permissions will be modified
+ * @return true if the permissions were successfully changed, false otherwise.
+ */
+ def chmod700(file: File): Boolean = {
+ file.setReadable(false, false) &&
+ file.setReadable(true, true) &&
+ file.setWritable(false, false) &&
+ file.setWritable(true, true) &&
+ file.setExecutable(false, false) &&
+ file.setExecutable(true, true)
+ }
+
/**
* Create a directory inside the given parent directory. The directory is guaranteed to be
* newly created, and is not marked for automatic deletion.
*/
- def createDirectory(root: String): File = {
+ def createDirectory(root: String, namePrefix: String = "spark"): File = {
var attempts = 0
- val maxAttempts = 10
+ val maxAttempts = MAX_DIR_CREATION_ATTEMPTS
var dir: File = null
while (dir == null) {
attempts += 1
@@ -261,7 +281,7 @@ private[spark] object Utils extends Logging {
maxAttempts + " attempts!")
}
try {
- dir = new File(root, "spark-" + UUID.randomUUID.toString)
+ dir = new File(root, namePrefix + "-" + UUID.randomUUID.toString)
if (dir.exists() || !dir.mkdirs()) {
dir = null
}
@@ -275,8 +295,10 @@ private[spark] object Utils extends Logging {
* Create a temporary directory inside the given parent directory. The directory will be
* automatically deleted when the VM shuts down.
*/
- def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = {
- val dir = createDirectory(root)
+ def createTempDir(
+ root: String = System.getProperty("java.io.tmpdir"),
+ namePrefix: String = "spark"): File = {
+ val dir = createDirectory(root, namePrefix)
registerShutdownDeleteDir(dir)
dir
}
@@ -359,8 +381,10 @@ private[spark] object Utils extends Logging {
}
/**
- * Download a file to target directory. Supports fetching the file in a variety of ways,
- * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter.
+ * Download a file or directory to target directory. Supports fetching the file in a variety of
+ * ways, including HTTP, Hadoop-compatible filesystems, and files on a standard filesystem, based
+ * on the URL parameter. Fetching directories is only supported from Hadoop-compatible
+ * filesystems.
*
* If `useCache` is true, first attempts to fetch the file to a local cache that's shared
* across executors running the same application. `useCache` is used mainly for
@@ -417,6 +441,12 @@ private[spark] object Utils extends Logging {
}
// Make the file executable - That's necessary for scripts
FileUtil.chmod(targetFile.getAbsolutePath, "a+x")
+
+ // Windows does not grant read permission by default to non-admin users
+ // Add read permission to owner explicitly
+ if (isWindows) {
+ FileUtil.chmod(targetFile.getAbsolutePath, "u+r")
+ }
}
/**
@@ -429,7 +459,6 @@ private[spark] object Utils extends Logging {
*
* @param url URL that `sourceFile` originated from, for logging purposes.
* @param in InputStream to download.
- * @param tempFile File path to download `in` to.
* @param destFile File path to move `tempFile` to.
* @param fileOverwrite Whether to delete/overwrite an existing `destFile` that does not match
* `sourceFile`
@@ -437,9 +466,11 @@ private[spark] object Utils extends Logging {
private def downloadFile(
url: String,
in: InputStream,
- tempFile: File,
destFile: File,
fileOverwrite: Boolean): Unit = {
+ val tempFile = File.createTempFile("fetchFileTemp", null,
+ new File(destFile.getParentFile.getAbsolutePath))
+ logInfo(s"Fetching $url to $tempFile")
try {
val out = new FileOutputStream(tempFile)
@@ -478,7 +509,7 @@ private[spark] object Utils extends Logging {
removeSourceFile: Boolean = false): Unit = {
if (destFile.exists) {
- if (!Files.equal(sourceFile, destFile)) {
+ if (!filesEqualRecursive(sourceFile, destFile)) {
if (fileOverwrite) {
logInfo(
s"File $destFile exists and does not match contents of $url, replacing it with $url"
@@ -513,13 +544,44 @@ private[spark] object Utils extends Logging {
Files.move(sourceFile, destFile)
} else {
logInfo(s"Copying ${sourceFile.getAbsolutePath} to ${destFile.getAbsolutePath}")
- Files.copy(sourceFile, destFile)
+ copyRecursive(sourceFile, destFile)
+ }
+ }
+
+ private def filesEqualRecursive(file1: File, file2: File): Boolean = {
+ if (file1.isDirectory && file2.isDirectory) {
+ val subfiles1 = file1.listFiles()
+ val subfiles2 = file2.listFiles()
+ if (subfiles1.size != subfiles2.size) {
+ return false
+ }
+ subfiles1.sortBy(_.getName).zip(subfiles2.sortBy(_.getName)).forall {
+ case (f1, f2) => filesEqualRecursive(f1, f2)
+ }
+ } else if (file1.isFile && file2.isFile) {
+ Files.equal(file1, file2)
+ } else {
+ false
+ }
+ }
+
+ private def copyRecursive(source: File, dest: File): Unit = {
+ if (source.isDirectory) {
+ if (!dest.mkdir()) {
+ throw new IOException(s"Failed to create directory ${dest.getPath}")
+ }
+ val subfiles = source.listFiles()
+ subfiles.foreach(f => copyRecursive(f, new File(dest, f.getName)))
+ } else {
+ Files.copy(source, dest)
}
}
/**
- * Download a file to target directory. Supports fetching the file in a variety of ways,
- * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter.
+ * Download a file or directory to target directory. Supports fetching the file in a variety of
+ * ways, including HTTP, Hadoop-compatible filesystems, and files on a standard filesystem, based
+ * on the URL parameter. Fetching directories is only supported from Hadoop-compatible
+ * filesystems.
*
* Throws SparkException if the target file already exists and has different contents than
* the requested file.
@@ -531,14 +593,11 @@ private[spark] object Utils extends Logging {
conf: SparkConf,
securityMgr: SecurityManager,
hadoopConf: Configuration) {
- val tempFile = File.createTempFile("fetchFileTemp", null, new File(targetDir.getAbsolutePath))
val targetFile = new File(targetDir, filename)
val uri = new URI(url)
val fileOverwrite = conf.getBoolean("spark.files.overwrite", defaultValue = false)
Option(uri.getScheme).getOrElse("file") match {
case "http" | "https" | "ftp" =>
- logInfo("Fetching " + url + " to " + tempFile)
-
var uc: URLConnection = null
if (securityMgr.isAuthenticationEnabled()) {
logDebug("fetchFile with security enabled")
@@ -549,23 +608,55 @@ private[spark] object Utils extends Logging {
logDebug("fetchFile not using security")
uc = new URL(url).openConnection()
}
+ Utils.setupSecureURLConnection(uc, securityMgr)
val timeout = conf.getInt("spark.files.fetchTimeout", 60) * 1000
uc.setConnectTimeout(timeout)
uc.setReadTimeout(timeout)
uc.connect()
val in = uc.getInputStream()
- downloadFile(url, in, tempFile, targetFile, fileOverwrite)
+ downloadFile(url, in, targetFile, fileOverwrite)
case "file" =>
// In the case of a local file, copy the local file to the target directory.
// Note the difference between uri vs url.
val sourceFile = if (uri.isAbsolute) new File(uri) else new File(url)
copyFile(url, sourceFile, targetFile, fileOverwrite)
case _ =>
- // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others
val fs = getHadoopFileSystem(uri, hadoopConf)
- val in = fs.open(new Path(uri))
- downloadFile(url, in, tempFile, targetFile, fileOverwrite)
+ val path = new Path(uri)
+ fetchHcfsFile(path, targetDir, fs, conf, hadoopConf, fileOverwrite,
+ filename = Some(filename))
+ }
+ }
+
+ /**
+ * Fetch a file or directory from a Hadoop-compatible filesystem.
+ *
+ * Visible for testing
+ */
+ private[spark] def fetchHcfsFile(
+ path: Path,
+ targetDir: File,
+ fs: FileSystem,
+ conf: SparkConf,
+ hadoopConf: Configuration,
+ fileOverwrite: Boolean,
+ filename: Option[String] = None): Unit = {
+ if (!targetDir.exists() && !targetDir.mkdir()) {
+ throw new IOException(s"Failed to create directory ${targetDir.getPath}")
+ }
+ val dest = new File(targetDir, filename.getOrElse(path.getName))
+ if (fs.isFile(path)) {
+ val in = fs.open(path)
+ try {
+ downloadFile(path.toString, in, dest, fileOverwrite)
+ } finally {
+ in.close()
+ }
+ } else {
+ fs.listStatus(path).foreach { fileStatus =>
+ fetchHcfsFile(fileStatus.getPath(), dest, fs, conf, hadoopConf, fileOverwrite)
+ }
}
}
@@ -597,28 +688,56 @@ private[spark] object Utils extends Logging {
* and returns only the directories that exist / could be created.
*
* If no directories could be created, this will return an empty list.
+ *
+ * This method will cache the local directories for the application when it's first invoked.
+ * So calling it multiple times with a different configuration will always return the same
+ * set of directories.
*/
private[spark] def getOrCreateLocalRootDirs(conf: SparkConf): Array[String] = {
- val confValue = if (isRunningInYarnContainer(conf)) {
+ if (localRootDirs == null) {
+ this.synchronized {
+ if (localRootDirs == null) {
+ localRootDirs = getOrCreateLocalRootDirsImpl(conf)
+ }
+ }
+ }
+ localRootDirs
+ }
+
+ private def getOrCreateLocalRootDirsImpl(conf: SparkConf): Array[String] = {
+ if (isRunningInYarnContainer(conf)) {
// If we are in yarn mode, systems can have different disk layouts so we must set it
- // to what Yarn on this system said was available.
- getYarnLocalDirs(conf)
+ // to what Yarn on this system said was available. Note this assumes that Yarn has
+ // created the directories already, and that they are secured so that only the
+ // user has access to them.
+ getYarnLocalDirs(conf).split(",")
+ } else if (conf.getenv("SPARK_EXECUTOR_DIRS") != null) {
+ conf.getenv("SPARK_EXECUTOR_DIRS").split(File.pathSeparator)
} else {
- Option(conf.getenv("SPARK_LOCAL_DIRS")).getOrElse(
- conf.get("spark.local.dir", System.getProperty("java.io.tmpdir")))
- }
- val rootDirs = confValue.split(',')
- logDebug(s"Getting/creating local root dirs at '$confValue'")
-
- rootDirs.flatMap { rootDir =>
- val localDir: File = new File(rootDir)
- val foundLocalDir = localDir.exists || localDir.mkdirs()
- if (!foundLocalDir) {
- logError(s"Failed to create local root dir in $rootDir. Ignoring this directory.")
- None
- } else {
- Some(rootDir)
- }
+ // In non-Yarn mode (or for the driver in yarn-client mode), we cannot trust the user
+ // configuration to point to a secure directory. So create a subdirectory with restricted
+ // permissions under each listed directory.
+ Option(conf.getenv("SPARK_LOCAL_DIRS"))
+ .getOrElse(conf.get("spark.local.dir", System.getProperty("java.io.tmpdir")))
+ .split(",")
+ .flatMap { root =>
+ try {
+ val rootDir = new File(root)
+ if (rootDir.exists || rootDir.mkdirs()) {
+ val dir = createTempDir(root)
+ chmod700(dir)
+ Some(dir.getAbsolutePath)
+ } else {
+ logError(s"Failed to create dir in $root. Ignoring this directory.")
+ None
+ }
+ } catch {
+ case e: IOException =>
+ logError(s"Failed to create local root dir in $root. Ignoring this directory.")
+ None
+ }
+ }
+ .toArray
}
}
@@ -637,6 +756,11 @@ private[spark] object Utils extends Logging {
localDirs
}
+ /** Used by unit tests. Do not call from other places. */
+ private[spark] def clearLocalRootDirs(): Unit = {
+ localRootDirs = null
+ }
+
/**
* Shuffle the elements of a collection into a random order, returning the
* result in a new collection. Unlike scala.util.Random.shuffle, this method
@@ -1067,9 +1191,9 @@ private[spark] object Utils extends Logging {
// finding the call site of a method.
val SPARK_CORE_CLASS_REGEX =
"""^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?(\.broadcast)?\.[A-Z]""".r
- val SCALA_CLASS_REGEX = """^scala""".r
+ val SCALA_CORE_CLASS_PREFIX = "scala"
val isSparkCoreClass = SPARK_CORE_CLASS_REGEX.findFirstIn(className).isDefined
- val isScalaClass = SCALA_CLASS_REGEX.findFirstIn(className).isDefined
+ val isScalaClass = className.startsWith(SCALA_CORE_CLASS_PREFIX)
// If the class is a Spark internal class or a Scala class, then exclude.
isSparkCoreClass || isScalaClass
}
@@ -1312,9 +1436,14 @@ private[spark] object Utils extends Logging {
hashAbs
}
- /** Returns a copy of the system properties that is thread-safe to iterator over. */
- def getSystemProperties(): Map[String, String] = {
- System.getProperties.clone().asInstanceOf[java.util.Properties].toMap[String, String]
+ /** Returns the system properties map that is thread-safe to iterator over. It gets the
+ * properties which have been set explicitly, as well as those for which only a default value
+ * has been defined. */
+ def getSystemProperties: Map[String, String] = {
+ val sysProps = for (key <- System.getProperties.stringPropertyNames()) yield
+ (key, System.getProperty(key))
+
+ sysProps.toMap
}
/**
@@ -1780,6 +1909,20 @@ private[spark] object Utils extends Logging {
PropertyConfigurator.configure(pro)
}
+ /**
+ * If the given URL connection is HttpsURLConnection, it sets the SSL socket factory and
+ * the host verifier from the given security manager.
+ */
+ def setupSecureURLConnection(urlConnection: URLConnection, sm: SecurityManager): URLConnection = {
+ urlConnection match {
+ case https: HttpsURLConnection =>
+ sm.sslSocketFactory.foreach(https.setSSLSocketFactory)
+ sm.hostnameVerifier.foreach(https.setHostnameVerifier)
+ https
+ case connection => connection
+ }
+ }
+
def invoke(
clazz: Class[_],
obj: AnyRef,
@@ -1872,6 +2015,16 @@ private[spark] object Utils extends Logging {
throw new SparkException("Invalid master URL: " + sparkUrl, e)
}
}
+
+ /**
+ * Returns the current user name. This is the currently logged in user, unless that's been
+ * overridden by the `SPARK_USER` environment variable.
+ */
+ def getCurrentUserName(): String = {
+ Option(System.getenv("SPARK_USER"))
+ .getOrElse(UserGroupInformation.getCurrentUser().getUserName())
+ }
+
}
/**
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
index 6ba03841f746b..d69f2d9048055 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
@@ -723,6 +723,7 @@ private[spark] class ExternalSorter[K, V, C](
partitionWriters.foreach(_.commitAndClose())
var out: FileOutputStream = null
var in: FileInputStream = null
+ val writeStartTime = System.nanoTime
try {
out = new FileOutputStream(outputFile, true)
for (i <- 0 until numPartitions) {
@@ -739,6 +740,8 @@ private[spark] class ExternalSorter[K, V, C](
if (in != null) {
in.close()
}
+ context.taskMetrics.shuffleWriteMetrics.foreach(
+ _.incShuffleWriteTime(System.nanoTime - writeStartTime))
}
} else {
// Either we're not bypassing merge-sort or we have only in-memory data; get an iterator by
@@ -763,6 +766,7 @@ private[spark] class ExternalSorter[K, V, C](
if (curWriteMetrics != null) {
m.incShuffleBytesWritten(curWriteMetrics.shuffleBytesWritten)
m.incShuffleWriteTime(curWriteMetrics.shuffleWriteTime)
+ m.incShuffleRecordsWritten(curWriteMetrics.shuffleRecordsWritten)
}
}
diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
index 7e76d060d6000..b6c380a8eea9f 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
@@ -71,12 +71,21 @@ class PrimitiveVector[@specialized(Long, Int, Double) V: ClassTag](initialSize:
/** Resizes the array, dropping elements if the total length decreases. */
def resize(newLength: Int): PrimitiveVector[V] = {
- val newArray = new Array[V](newLength)
- _array.copyToArray(newArray)
- _array = newArray
+ _array = copyArrayWithLength(newLength)
if (newLength < _numElements) {
_numElements = newLength
}
this
}
+
+ /** Return a trimmed version of the underlying array. */
+ def toArray: Array[V] = {
+ copyArrayWithLength(size)
+ }
+
+ private def copyArrayWithLength(length: Int): Array[V] = {
+ val copy = new Array[V](length)
+ _array.copyToArray(copy)
+ copy
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala
index 65a7b4e0d497b..dfcfb66af8613 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala
@@ -36,11 +36,4 @@ private[spark] class SizeTrackingVector[T: ClassTag]
resetSamples()
this
}
-
- /**
- * Return a trimmed version of the underlying array.
- */
- def toArray: Array[T] = {
- super.iterator.toArray
- }
}
diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala
index 9f54312074856..747ecf075a397 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala
@@ -42,9 +42,6 @@ private[spark] trait Spillable[C] extends Logging {
// Memory manager that can be used to acquire/release memory
private[this] val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager
- // Threshold for `elementsRead` before we start tracking this collection's memory usage
- private[this] val trackMemoryThreshold = 1000
-
// Initial threshold for the size of a collection before we start tracking its memory usage
// Exposed for testing
private[this] val initialMemoryThreshold: Long =
@@ -72,8 +69,7 @@ private[spark] trait Spillable[C] extends Logging {
* @return true if `collection` was spilled to disk; false otherwise
*/
protected def maybeSpill(collection: C, currentMemory: Long): Boolean = {
- if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 &&
- currentMemory >= myMemoryThreshold) {
+ if (elementsRead % 32 == 0 && currentMemory >= myMemoryThreshold) {
// Claim up to double our current memory from the shuffle memory pool
val amountToRequest = 2 * currentMemory - myMemoryThreshold
val granted = shuffleMemoryManager.tryToAcquire(amountToRequest)
diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java
index b16a1e9460286..74e88c767ee07 100644
--- a/core/src/test/java/org/apache/spark/JavaAPISuite.java
+++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java
@@ -708,6 +708,10 @@ public void javaDoubleRDDHistoGram() {
// Test with provided buckets
long[] histogram = rdd.histogram(expected_buckets);
Assert.assertArrayEquals(expected_counts, histogram);
+ // SPARK-5744
+ Assert.assertArrayEquals(
+ new long[] {0},
+ sc.parallelizeDoubles(new ArrayList(0), 1).histogram(new double[]{0.0, 1.0}));
}
@Test
diff --git a/core/src/test/java/org/apache/spark/util/collection/TestTimSort.java b/core/src/test/java/org/apache/spark/util/collection/TestTimSort.java
new file mode 100644
index 0000000000000..45772b6d3c20d
--- /dev/null
+++ b/core/src/test/java/org/apache/spark/util/collection/TestTimSort.java
@@ -0,0 +1,134 @@
+/**
+ * Copyright 2015 Stijn de Gouw
+ *
+ * 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.
+ *
+ */
+package org.apache.spark.util.collection;
+
+import java.util.*;
+
+/**
+ * This codes generates a int array which fails the standard TimSort.
+ *
+ * The blog that reported the bug
+ * http://www.envisage-project.eu/timsort-specification-and-verification/
+ *
+ * This codes was originally wrote by Stijn de Gouw, modified by Evan Yu to adapt to
+ * our test suite.
+ *
+ * https://github.com/abstools/java-timsort-bug
+ * https://github.com/abstools/java-timsort-bug/blob/master/LICENSE
+ */
+public class TestTimSort {
+
+ private static final int MIN_MERGE = 32;
+
+ /**
+ * Returns an array of integers that demonstrate the bug in TimSort
+ */
+ public static int[] getTimSortBugTestSet(int length) {
+ int minRun = minRunLength(length);
+ List runs = runsJDKWorstCase(minRun, length);
+ return createArray(runs, length);
+ }
+
+ private static int minRunLength(int n) {
+ int r = 0; // Becomes 1 if any 1 bits are shifted off
+ while (n >= MIN_MERGE) {
+ r |= (n & 1);
+ n >>= 1;
+ }
+ return n + r;
+ }
+
+ private static int[] createArray(List runs, int length) {
+ int[] a = new int[length];
+ Arrays.fill(a, 0);
+ int endRun = -1;
+ for (long len : runs) {
+ a[endRun += len] = 1;
+ }
+ a[length - 1] = 0;
+ return a;
+ }
+
+ /**
+ * Fills runs
with a sequence of run lengths of the form
+ * Y_n x_{n,1} x_{n,2} ... x_{n,l_n}
+ * Y_{n-1} x_{n-1,1} x_{n-1,2} ... x_{n-1,l_{n-1}}
+ * ...
+ * Y_1 x_{1,1} x_{1,2} ... x_{1,l_1}
+ * The Y_i's are chosen to satisfy the invariant throughout execution,
+ * but the x_{i,j}'s are merged (by TimSort.mergeCollapse
)
+ * into an X_i that violates the invariant.
+ *
+ * @param length The sum of all run lengths that will be added to runs
.
+ */
+ private static List runsJDKWorstCase(int minRun, int length) {
+ List runs = new ArrayList();
+
+ long runningTotal = 0, Y = minRun + 4, X = minRun;
+
+ while (runningTotal + Y + X <= length) {
+ runningTotal += X + Y;
+ generateJDKWrongElem(runs, minRun, X);
+ runs.add(0, Y);
+ // X_{i+1} = Y_i + x_{i,1} + 1, since runs.get(1) = x_{i,1}
+ X = Y + runs.get(1) + 1;
+ // Y_{i+1} = X_{i+1} + Y_i + 1
+ Y += X + 1;
+ }
+
+ if (runningTotal + X <= length) {
+ runningTotal += X;
+ generateJDKWrongElem(runs, minRun, X);
+ }
+
+ runs.add(length - runningTotal);
+ return runs;
+ }
+
+ /**
+ * Adds a sequence x_1, ..., x_n of run lengths to runs
such that:
+ * 1. X = x_1 + ... + x_n
+ * 2. x_j >= minRun for all j
+ * 3. x_1 + ... + x_{j-2} < x_j < x_1 + ... + x_{j-1} for all j
+ * These conditions guarantee that TimSort merges all x_j's one by one
+ * (resulting in X) using only merges on the second-to-last element.
+ *
+ * @param X The sum of the sequence that should be added to runs.
+ */
+ private static void generateJDKWrongElem(List runs, int minRun, long X) {
+ for (long newTotal; X >= 2 * minRun + 1; X = newTotal) {
+ //Default strategy
+ newTotal = X / 2 + 1;
+ //Specialized strategies
+ if (3 * minRun + 3 <= X && X <= 4 * minRun + 1) {
+ // add x_1=MIN+1, x_2=MIN, x_3=X-newTotal to runs
+ newTotal = 2 * minRun + 1;
+ } else if (5 * minRun + 5 <= X && X <= 6 * minRun + 5) {
+ // add x_1=MIN+1, x_2=MIN, x_3=MIN+2, x_4=X-newTotal to runs
+ newTotal = 3 * minRun + 3;
+ } else if (8 * minRun + 9 <= X && X <= 10 * minRun + 9) {
+ // add x_1=MIN+1, x_2=MIN, x_3=MIN+2, x_4=2MIN+2, x_5=X-newTotal to runs
+ newTotal = 5 * minRun + 5;
+ } else if (13 * minRun + 15 <= X && X <= 16 * minRun + 17) {
+ // add x_1=MIN+1, x_2=MIN, x_3=MIN+2, x_4=2MIN+2, x_5=3MIN+4, x_6=X-newTotal to runs
+ newTotal = 8 * minRun + 9;
+ }
+ runs.add(0, X - newTotal);
+ }
+ runs.add(0, X);
+ }
+}
diff --git a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java b/core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java
similarity index 93%
rename from core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java
rename to core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java
index e9ec700e32e15..e38bc38949d7c 100644
--- a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java
+++ b/core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java
@@ -15,9 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark.util;
+package test.org.apache.spark;
import org.apache.spark.TaskContext;
+import org.apache.spark.util.TaskCompletionListener;
/**
diff --git a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java
new file mode 100644
index 0000000000000..4a918f725dc91
--- /dev/null
+++ b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package test.org.apache.spark;
+
+import org.apache.spark.TaskContext;
+
+/**
+ * Something to make sure that TaskContext can be used in Java.
+ */
+public class JavaTaskContextCompileCheck {
+
+ public static void test() {
+ TaskContext tc = TaskContext.get();
+
+ tc.isCompleted();
+ tc.isInterrupted();
+ tc.isRunningLocally();
+
+ tc.addTaskCompletionListener(new JavaTaskCompletionListenerImpl());
+
+ tc.attemptNumber();
+ tc.partitionId();
+ tc.stageId();
+ tc.taskAttemptId();
+ }
+}
diff --git a/core/src/test/resources/keystore b/core/src/test/resources/keystore
new file mode 100644
index 0000000000000..f8310e39ba1e0
Binary files /dev/null and b/core/src/test/resources/keystore differ
diff --git a/core/src/test/resources/test_metrics_system.properties b/core/src/test/resources/test_metrics_system.properties
index 35d0bd3b8d0b8..4e8b8465696e5 100644
--- a/core/src/test/resources/test_metrics_system.properties
+++ b/core/src/test/resources/test_metrics_system.properties
@@ -18,7 +18,5 @@
*.sink.console.period = 10
*.sink.console.unit = seconds
test.sink.console.class = org.apache.spark.metrics.sink.ConsoleSink
-test.sink.dummy.class = org.apache.spark.metrics.sink.DummySink
-test.source.dummy.class = org.apache.spark.metrics.source.DummySource
test.sink.console.period = 20
test.sink.console.unit = minutes
diff --git a/core/src/test/resources/truststore b/core/src/test/resources/truststore
new file mode 100644
index 0000000000000..a6b1d46e1f391
Binary files /dev/null and b/core/src/test/resources/truststore differ
diff --git a/core/src/test/resources/untrusted-keystore b/core/src/test/resources/untrusted-keystore
new file mode 100644
index 0000000000000..6015b02caa128
Binary files /dev/null and b/core/src/test/resources/untrusted-keystore differ
diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
index f087fc550dde3..bd0f8bdefa171 100644
--- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
@@ -18,6 +18,7 @@
package org.apache.spark
import scala.collection.mutable
+import scala.ref.WeakReference
import org.scalatest.FunSuite
import org.scalatest.Matchers
@@ -136,4 +137,23 @@ class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext {
}
}
+ test ("garbage collection") {
+ // Create an accumulator and let it go out of scope to test that it's properly garbage collected
+ sc = new SparkContext("local", "test")
+ var acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]())
+ val accId = acc.id
+ val ref = WeakReference(acc)
+
+ // Ensure the accumulator is present
+ assert(ref.get.isDefined)
+
+ // Remove the explicit reference to it and allow weak reference to get garbage collected
+ acc = null
+ System.gc()
+ assert(ref.get.isEmpty)
+
+ Accumulators.remove(accId)
+ assert(!Accumulators.originals.get(accId).isDefined)
+ }
+
}
diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala
index d7d9dc7b50f30..4b25c200a695a 100644
--- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala
@@ -17,16 +17,18 @@
package org.apache.spark
+import org.mockito.Mockito._
import org.scalatest.{BeforeAndAfter, FunSuite}
-import org.scalatest.mock.EasyMockSugar
+import org.scalatest.mock.MockitoSugar
-import org.apache.spark.executor.{DataReadMethod, TaskMetrics}
+import org.apache.spark.executor.DataReadMethod
import org.apache.spark.rdd.RDD
import org.apache.spark.storage._
// TODO: Test the CacheManager's thread-safety aspects
-class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar {
- var sc : SparkContext = _
+class CacheManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAfter
+ with MockitoSugar {
+
var blockManager: BlockManager = _
var cacheManager: CacheManager = _
var split: Partition = _
@@ -57,10 +59,6 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
}.cache()
}
- after {
- sc.stop()
- }
-
test("get uncached rdd") {
// Do not mock this test, because attempting to match Array[Any], which is not covariant,
// in blockManager.put is a losing battle. You have been warned.
@@ -75,29 +73,21 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
}
test("get cached rdd") {
- expecting {
- val result = new BlockResult(Array(5, 6, 7).iterator, DataReadMethod.Memory, 12)
- blockManager.get(RDDBlockId(0, 0)).andReturn(Some(result))
- }
+ val result = new BlockResult(Array(5, 6, 7).iterator, DataReadMethod.Memory, 12)
+ when(blockManager.get(RDDBlockId(0, 0))).thenReturn(Some(result))
- whenExecuting(blockManager) {
- val context = new TaskContextImpl(0, 0, 0, 0)
- val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
- assert(value.toList === List(5, 6, 7))
- }
+ val context = new TaskContextImpl(0, 0, 0, 0)
+ val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
+ assert(value.toList === List(5, 6, 7))
}
test("get uncached local rdd") {
- expecting {
- // Local computation should not persist the resulting value, so don't expect a put().
- blockManager.get(RDDBlockId(0, 0)).andReturn(None)
- }
+ // Local computation should not persist the resulting value, so don't expect a put().
+ when(blockManager.get(RDDBlockId(0, 0))).thenReturn(None)
- whenExecuting(blockManager) {
- val context = new TaskContextImpl(0, 0, 0, 0, true)
- val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
- assert(value.toList === List(1, 2, 3, 4))
- }
+ val context = new TaskContextImpl(0, 0, 0, 0, true)
+ val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
+ assert(value.toList === List(1, 2, 3, 4))
}
test("verify task metrics updated correctly") {
diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala
index ae2ae7ed0d3aa..cdfaacee7da40 100644
--- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala
@@ -382,6 +382,10 @@ class CleanerTester(
toBeCleanedBroadcstIds -= broadcastId
logInfo("Broadcast" + broadcastId + " cleaned")
}
+
+ def accumCleaned(accId: Long): Unit = {
+ logInfo("Cleaned accId " + accId + " cleaned")
+ }
}
val MAX_VALIDATION_ATTEMPTS = 10
diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
index 0e4df17c1bf87..abfcee75728dc 100644
--- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
@@ -22,7 +22,8 @@ import scala.collection.mutable
import org.scalatest.{FunSuite, PrivateMethodTester}
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.scheduler._
-import org.apache.spark.storage.BlockManagerId
+import org.apache.spark.scheduler.cluster.ExecutorInfo
+import org.apache.spark.util.ManualClock
/**
* Test add and remove behavior of ExecutorAllocationManager.
@@ -32,24 +33,23 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
import ExecutorAllocationManagerSuite._
test("verify min/max executors") {
- // No min or max
val conf = new SparkConf()
.setMaster("local")
.setAppName("test-executor-allocation-manager")
.set("spark.dynamicAllocation.enabled", "true")
.set("spark.dynamicAllocation.testing", "true")
- intercept[SparkException] { new SparkContext(conf) }
- SparkEnv.get.stop() // cleanup the created environment
- SparkContext.clearActiveContext()
+ val sc0 = new SparkContext(conf)
+ assert(sc0.executorAllocationManager.isDefined)
+ sc0.stop()
- // Only min
- val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1")
+ // Min < 0
+ val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "-1")
intercept[SparkException] { new SparkContext(conf1) }
SparkEnv.get.stop()
SparkContext.clearActiveContext()
- // Only max
- val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2")
+ // Max < 0
+ val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "-1")
intercept[SparkException] { new SparkContext(conf2) }
SparkEnv.get.stop()
SparkContext.clearActiveContext()
@@ -145,8 +145,8 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
// Verify that running a task reduces the cap
sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, 3)))
- sc.listenerBus.postToAll(SparkListenerBlockManagerAdded(
- 0L, BlockManagerId("executor-1", "host1", 1), 100L))
+ sc.listenerBus.postToAll(SparkListenerExecutorAdded(
+ 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty)))
sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1")))
assert(numExecutorsPending(manager) === 4)
assert(addExecutors(manager) === 1)
@@ -176,6 +176,33 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
assert(numExecutorsPending(manager) === 9)
}
+ test("cancel pending executors when no longer needed") {
+ sc = createSparkContext(1, 10)
+ val manager = sc.executorAllocationManager.get
+ sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(2, 5)))
+
+ assert(numExecutorsPending(manager) === 0)
+ assert(numExecutorsToAdd(manager) === 1)
+ assert(addExecutors(manager) === 1)
+ assert(numExecutorsPending(manager) === 1)
+ assert(numExecutorsToAdd(manager) === 2)
+ assert(addExecutors(manager) === 2)
+ assert(numExecutorsPending(manager) === 3)
+
+ val task1Info = createTaskInfo(0, 0, "executor-1")
+ sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, task1Info))
+
+ assert(numExecutorsToAdd(manager) === 4)
+ assert(addExecutors(manager) === 2)
+
+ val task2Info = createTaskInfo(1, 0, "executor-1")
+ sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, task2Info))
+ sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, null, task1Info, null))
+ sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, null, task2Info, null))
+
+ assert(adjustRequestedExecutors(manager) === -1)
+ }
+
test("remove executors") {
sc = createSparkContext(5, 10)
val manager = sc.executorAllocationManager.get
@@ -271,15 +298,15 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
assert(removeExecutor(manager, "5"))
assert(removeExecutor(manager, "6"))
assert(executorIds(manager).size === 10)
- assert(addExecutors(manager) === 0) // still at upper limit
+ assert(addExecutors(manager) === 1)
onExecutorRemoved(manager, "3")
onExecutorRemoved(manager, "4")
assert(executorIds(manager).size === 8)
// Add succeeds again, now that we are no longer at the upper limit
// Number of executors added restarts at 1
- assert(addExecutors(manager) === 1)
- assert(addExecutors(manager) === 1) // upper limit reached again
+ assert(addExecutors(manager) === 2)
+ assert(addExecutors(manager) === 1) // upper limit reached
assert(addExecutors(manager) === 0)
assert(executorIds(manager).size === 8)
onExecutorRemoved(manager, "5")
@@ -287,9 +314,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
onExecutorAdded(manager, "13")
onExecutorAdded(manager, "14")
assert(executorIds(manager).size === 8)
- assert(addExecutors(manager) === 1)
- assert(addExecutors(manager) === 1) // upper limit reached again
- assert(addExecutors(manager) === 0)
+ assert(addExecutors(manager) === 0) // still at upper limit
onExecutorAdded(manager, "15")
onExecutorAdded(manager, "16")
assert(executorIds(manager).size === 10)
@@ -297,7 +322,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
test("starting/canceling add timer") {
sc = createSparkContext(2, 10)
- val clock = new TestClock(8888L)
+ val clock = new ManualClock(8888L)
val manager = sc.executorAllocationManager.get
manager.setClock(clock)
@@ -306,21 +331,21 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
onSchedulerBacklogged(manager)
val firstAddTime = addTime(manager)
assert(firstAddTime === clock.getTimeMillis + schedulerBacklogTimeout * 1000)
- clock.tick(100L)
+ clock.advance(100L)
onSchedulerBacklogged(manager)
assert(addTime(manager) === firstAddTime) // timer is already started
- clock.tick(200L)
+ clock.advance(200L)
onSchedulerBacklogged(manager)
assert(addTime(manager) === firstAddTime)
onSchedulerQueueEmpty(manager)
// Restart add timer
- clock.tick(1000L)
+ clock.advance(1000L)
assert(addTime(manager) === NOT_SET)
onSchedulerBacklogged(manager)
val secondAddTime = addTime(manager)
assert(secondAddTime === clock.getTimeMillis + schedulerBacklogTimeout * 1000)
- clock.tick(100L)
+ clock.advance(100L)
onSchedulerBacklogged(manager)
assert(addTime(manager) === secondAddTime) // timer is already started
assert(addTime(manager) !== firstAddTime)
@@ -329,7 +354,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
test("starting/canceling remove timers") {
sc = createSparkContext(2, 10)
- val clock = new TestClock(14444L)
+ val clock = new ManualClock(14444L)
val manager = sc.executorAllocationManager.get
manager.setClock(clock)
@@ -342,17 +367,17 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
assert(removeTimes(manager).contains("1"))
val firstRemoveTime = removeTimes(manager)("1")
assert(firstRemoveTime === clock.getTimeMillis + executorIdleTimeout * 1000)
- clock.tick(100L)
+ clock.advance(100L)
onExecutorIdle(manager, "1")
assert(removeTimes(manager)("1") === firstRemoveTime) // timer is already started
- clock.tick(200L)
+ clock.advance(200L)
onExecutorIdle(manager, "1")
assert(removeTimes(manager)("1") === firstRemoveTime)
- clock.tick(300L)
+ clock.advance(300L)
onExecutorIdle(manager, "2")
assert(removeTimes(manager)("2") !== firstRemoveTime) // different executor
assert(removeTimes(manager)("2") === clock.getTimeMillis + executorIdleTimeout * 1000)
- clock.tick(400L)
+ clock.advance(400L)
onExecutorIdle(manager, "3")
assert(removeTimes(manager)("3") !== firstRemoveTime)
assert(removeTimes(manager)("3") === clock.getTimeMillis + executorIdleTimeout * 1000)
@@ -361,7 +386,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
assert(removeTimes(manager).contains("3"))
// Restart remove timer
- clock.tick(1000L)
+ clock.advance(1000L)
onExecutorBusy(manager, "1")
assert(removeTimes(manager).size === 2)
onExecutorIdle(manager, "1")
@@ -377,7 +402,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
test("mock polling loop with no events") {
sc = createSparkContext(1, 20)
val manager = sc.executorAllocationManager.get
- val clock = new TestClock(2020L)
+ val clock = new ManualClock(2020L)
manager.setClock(clock)
// No events - we should not be adding or removing
@@ -386,15 +411,15 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
schedule(manager)
assert(numExecutorsPending(manager) === 0)
assert(executorsPendingToRemove(manager).isEmpty)
- clock.tick(100L)
+ clock.advance(100L)
schedule(manager)
assert(numExecutorsPending(manager) === 0)
assert(executorsPendingToRemove(manager).isEmpty)
- clock.tick(1000L)
+ clock.advance(1000L)
schedule(manager)
assert(numExecutorsPending(manager) === 0)
assert(executorsPendingToRemove(manager).isEmpty)
- clock.tick(10000L)
+ clock.advance(10000L)
schedule(manager)
assert(numExecutorsPending(manager) === 0)
assert(executorsPendingToRemove(manager).isEmpty)
@@ -402,57 +427,57 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
test("mock polling loop add behavior") {
sc = createSparkContext(1, 20)
- val clock = new TestClock(2020L)
+ val clock = new ManualClock(2020L)
val manager = sc.executorAllocationManager.get
manager.setClock(clock)
sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 1000)))
// Scheduler queue backlogged
onSchedulerBacklogged(manager)
- clock.tick(schedulerBacklogTimeout * 1000 / 2)
+ clock.advance(schedulerBacklogTimeout * 1000 / 2)
schedule(manager)
assert(numExecutorsPending(manager) === 0) // timer not exceeded yet
- clock.tick(schedulerBacklogTimeout * 1000)
+ clock.advance(schedulerBacklogTimeout * 1000)
schedule(manager)
assert(numExecutorsPending(manager) === 1) // first timer exceeded
- clock.tick(sustainedSchedulerBacklogTimeout * 1000 / 2)
+ clock.advance(sustainedSchedulerBacklogTimeout * 1000 / 2)
schedule(manager)
assert(numExecutorsPending(manager) === 1) // second timer not exceeded yet
- clock.tick(sustainedSchedulerBacklogTimeout * 1000)
+ clock.advance(sustainedSchedulerBacklogTimeout * 1000)
schedule(manager)
assert(numExecutorsPending(manager) === 1 + 2) // second timer exceeded
- clock.tick(sustainedSchedulerBacklogTimeout * 1000)
+ clock.advance(sustainedSchedulerBacklogTimeout * 1000)
schedule(manager)
assert(numExecutorsPending(manager) === 1 + 2 + 4) // third timer exceeded
// Scheduler queue drained
onSchedulerQueueEmpty(manager)
- clock.tick(sustainedSchedulerBacklogTimeout * 1000)
+ clock.advance(sustainedSchedulerBacklogTimeout * 1000)
schedule(manager)
assert(numExecutorsPending(manager) === 7) // timer is canceled
- clock.tick(sustainedSchedulerBacklogTimeout * 1000)
+ clock.advance(sustainedSchedulerBacklogTimeout * 1000)
schedule(manager)
assert(numExecutorsPending(manager) === 7)
// Scheduler queue backlogged again
onSchedulerBacklogged(manager)
- clock.tick(schedulerBacklogTimeout * 1000)
+ clock.advance(schedulerBacklogTimeout * 1000)
schedule(manager)
assert(numExecutorsPending(manager) === 7 + 1) // timer restarted
- clock.tick(sustainedSchedulerBacklogTimeout * 1000)
+ clock.advance(sustainedSchedulerBacklogTimeout * 1000)
schedule(manager)
assert(numExecutorsPending(manager) === 7 + 1 + 2)
- clock.tick(sustainedSchedulerBacklogTimeout * 1000)
+ clock.advance(sustainedSchedulerBacklogTimeout * 1000)
schedule(manager)
assert(numExecutorsPending(manager) === 7 + 1 + 2 + 4)
- clock.tick(sustainedSchedulerBacklogTimeout * 1000)
+ clock.advance(sustainedSchedulerBacklogTimeout * 1000)
schedule(manager)
assert(numExecutorsPending(manager) === 20) // limit reached
}
test("mock polling loop remove behavior") {
sc = createSparkContext(1, 20)
- val clock = new TestClock(2020L)
+ val clock = new ManualClock(2020L)
val manager = sc.executorAllocationManager.get
manager.setClock(clock)
@@ -462,11 +487,11 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
onExecutorAdded(manager, "executor-3")
assert(removeTimes(manager).size === 3)
assert(executorsPendingToRemove(manager).isEmpty)
- clock.tick(executorIdleTimeout * 1000 / 2)
+ clock.advance(executorIdleTimeout * 1000 / 2)
schedule(manager)
assert(removeTimes(manager).size === 3) // idle threshold not reached yet
assert(executorsPendingToRemove(manager).isEmpty)
- clock.tick(executorIdleTimeout * 1000)
+ clock.advance(executorIdleTimeout * 1000)
schedule(manager)
assert(removeTimes(manager).isEmpty) // idle threshold exceeded
assert(executorsPendingToRemove(manager).size === 2) // limit reached (1 executor remaining)
@@ -487,7 +512,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
assert(!removeTimes(manager).contains("executor-5"))
assert(!removeTimes(manager).contains("executor-6"))
assert(executorsPendingToRemove(manager).size === 2)
- clock.tick(executorIdleTimeout * 1000)
+ clock.advance(executorIdleTimeout * 1000)
schedule(manager)
assert(removeTimes(manager).isEmpty) // idle executors are removed
assert(executorsPendingToRemove(manager).size === 4)
@@ -505,7 +530,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
assert(removeTimes(manager).contains("executor-5"))
assert(removeTimes(manager).contains("executor-6"))
assert(executorsPendingToRemove(manager).size === 4)
- clock.tick(executorIdleTimeout * 1000)
+ clock.advance(executorIdleTimeout * 1000)
schedule(manager)
assert(removeTimes(manager).isEmpty)
assert(executorsPendingToRemove(manager).size === 6) // limit reached (1 executor remaining)
@@ -579,30 +604,28 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
assert(removeTimes(manager).isEmpty)
// New executors have registered
- sc.listenerBus.postToAll(SparkListenerBlockManagerAdded(
- 0L, BlockManagerId("executor-1", "host1", 1), 100L))
+ sc.listenerBus.postToAll(SparkListenerExecutorAdded(
+ 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty)))
assert(executorIds(manager).size === 1)
assert(executorIds(manager).contains("executor-1"))
assert(removeTimes(manager).size === 1)
assert(removeTimes(manager).contains("executor-1"))
- sc.listenerBus.postToAll(SparkListenerBlockManagerAdded(
- 0L, BlockManagerId("executor-2", "host2", 1), 100L))
+ sc.listenerBus.postToAll(SparkListenerExecutorAdded(
+ 0L, "executor-2", new ExecutorInfo("host2", 1, Map.empty)))
assert(executorIds(manager).size === 2)
assert(executorIds(manager).contains("executor-2"))
assert(removeTimes(manager).size === 2)
assert(removeTimes(manager).contains("executor-2"))
// Existing executors have disconnected
- sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved(
- 0L, BlockManagerId("executor-1", "host1", 1)))
+ sc.listenerBus.postToAll(SparkListenerExecutorRemoved(0L, "executor-1", ""))
assert(executorIds(manager).size === 1)
assert(!executorIds(manager).contains("executor-1"))
assert(removeTimes(manager).size === 1)
assert(!removeTimes(manager).contains("executor-1"))
// Unknown executor has disconnected
- sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved(
- 0L, BlockManagerId("executor-3", "host3", 1)))
+ sc.listenerBus.postToAll(SparkListenerExecutorRemoved(0L, "executor-3", ""))
assert(executorIds(manager).size === 1)
assert(removeTimes(manager).size === 1)
}
@@ -614,8 +637,8 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
assert(removeTimes(manager).isEmpty)
sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1")))
- sc.listenerBus.postToAll(SparkListenerBlockManagerAdded(
- 0L, BlockManagerId("executor-1", "host1", 1), 100L))
+ sc.listenerBus.postToAll(SparkListenerExecutorAdded(
+ 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty)))
assert(executorIds(manager).size === 1)
assert(executorIds(manager).contains("executor-1"))
assert(removeTimes(manager).size === 0)
@@ -626,16 +649,16 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
val manager = sc.executorAllocationManager.get
assert(executorIds(manager).isEmpty)
assert(removeTimes(manager).isEmpty)
- sc.listenerBus.postToAll(SparkListenerBlockManagerAdded(
- 0L, BlockManagerId("executor-1", "host1", 1), 100L))
+ sc.listenerBus.postToAll(SparkListenerExecutorAdded(
+ 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty)))
sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1")))
assert(executorIds(manager).size === 1)
assert(executorIds(manager).contains("executor-1"))
assert(removeTimes(manager).size === 0)
- sc.listenerBus.postToAll(SparkListenerBlockManagerAdded(
- 0L, BlockManagerId("executor-2", "host1", 1), 100L))
+ sc.listenerBus.postToAll(SparkListenerExecutorAdded(
+ 0L, "executor-2", new ExecutorInfo("host1", 1, Map.empty)))
assert(executorIds(manager).size === 2)
assert(executorIds(manager).contains("executor-2"))
assert(removeTimes(manager).size === 1)
@@ -682,6 +705,7 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester {
private val _numExecutorsToAdd = PrivateMethod[Int]('numExecutorsToAdd)
private val _numExecutorsPending = PrivateMethod[Int]('numExecutorsPending)
+ private val _maxNumExecutorsNeeded = PrivateMethod[Int]('maxNumExecutorsNeeded)
private val _executorsPendingToRemove =
PrivateMethod[collection.Set[String]]('executorsPendingToRemove)
private val _executorIds = PrivateMethod[collection.Set[String]]('executorIds)
@@ -689,6 +713,7 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester {
private val _removeTimes = PrivateMethod[collection.Map[String, Long]]('removeTimes)
private val _schedule = PrivateMethod[Unit]('schedule)
private val _addExecutors = PrivateMethod[Int]('addExecutors)
+ private val _addOrCancelExecutorRequests = PrivateMethod[Int]('addOrCancelExecutorRequests)
private val _removeExecutor = PrivateMethod[Boolean]('removeExecutor)
private val _onExecutorAdded = PrivateMethod[Unit]('onExecutorAdded)
private val _onExecutorRemoved = PrivateMethod[Unit]('onExecutorRemoved)
@@ -727,7 +752,12 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester {
}
private def addExecutors(manager: ExecutorAllocationManager): Int = {
- manager invokePrivate _addExecutors()
+ val maxNumExecutorsNeeded = manager invokePrivate _maxNumExecutorsNeeded()
+ manager invokePrivate _addExecutors(maxNumExecutorsNeeded)
+ }
+
+ private def adjustRequestedExecutors(manager: ExecutorAllocationManager): Int = {
+ manager invokePrivate _addOrCancelExecutorRequests(0L)
}
private def removeExecutor(manager: ExecutorAllocationManager, id: String): Boolean = {
diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala
index 0f49ce4754fbb..5fdf6bc2777e3 100644
--- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala
@@ -18,13 +18,19 @@
package org.apache.spark
import java.io._
+import java.net.URI
import java.util.jar.{JarEntry, JarOutputStream}
+import javax.net.ssl.SSLHandshakeException
import com.google.common.io.ByteStreams
+import org.apache.commons.io.{FileUtils, IOUtils}
+import org.apache.commons.lang3.RandomUtils
import org.scalatest.FunSuite
import org.apache.spark.util.Utils
+import SSLSampleConfigs._
+
class FileServerSuite extends FunSuite with LocalSparkContext {
@transient var tmpDir: File = _
@@ -168,4 +174,88 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
}
}
+ test ("HttpFileServer should work with SSL") {
+ val sparkConf = sparkSSLConfig()
+ val sm = new SecurityManager(sparkConf)
+ val server = new HttpFileServer(sparkConf, sm, 0)
+ try {
+ server.initialize()
+
+ fileTransferTest(server, sm)
+ } finally {
+ server.stop()
+ }
+ }
+
+ test ("HttpFileServer should work with SSL and good credentials") {
+ val sparkConf = sparkSSLConfig()
+ sparkConf.set("spark.authenticate", "true")
+ sparkConf.set("spark.authenticate.secret", "good")
+
+ val sm = new SecurityManager(sparkConf)
+ val server = new HttpFileServer(sparkConf, sm, 0)
+ try {
+ server.initialize()
+
+ fileTransferTest(server, sm)
+ } finally {
+ server.stop()
+ }
+ }
+
+ test ("HttpFileServer should not work with valid SSL and bad credentials") {
+ val sparkConf = sparkSSLConfig()
+ sparkConf.set("spark.authenticate", "true")
+ sparkConf.set("spark.authenticate.secret", "bad")
+
+ val sm = new SecurityManager(sparkConf)
+ val server = new HttpFileServer(sparkConf, sm, 0)
+ try {
+ server.initialize()
+
+ intercept[IOException] {
+ fileTransferTest(server)
+ }
+ } finally {
+ server.stop()
+ }
+ }
+
+ test ("HttpFileServer should not work with SSL when the server is untrusted") {
+ val sparkConf = sparkSSLConfigUntrusted()
+ val sm = new SecurityManager(sparkConf)
+ val server = new HttpFileServer(sparkConf, sm, 0)
+ try {
+ server.initialize()
+
+ intercept[SSLHandshakeException] {
+ fileTransferTest(server)
+ }
+ } finally {
+ server.stop()
+ }
+ }
+
+ def fileTransferTest(server: HttpFileServer, sm: SecurityManager = null): Unit = {
+ val randomContent = RandomUtils.nextBytes(100)
+ val file = File.createTempFile("FileServerSuite", "sslTests", tmpDir)
+ FileUtils.writeByteArrayToFile(file, randomContent)
+ server.addFile(file)
+
+ val uri = new URI(server.serverUri + "/files/" + file.getName)
+
+ val connection = if (sm != null && sm.isAuthenticationEnabled()) {
+ Utils.constructURIForAuthentication(uri, sm).toURL.openConnection()
+ } else {
+ uri.toURL.openConnection()
+ }
+
+ if (sm != null) {
+ Utils.setupSecureURLConnection(connection, sm)
+ }
+
+ val buf = IOUtils.toByteArray(connection.getInputStream)
+ assert(buf === randomContent)
+ }
+
}
diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala
index 5e24196101fbc..7acd27c735727 100644
--- a/core/src/test/scala/org/apache/spark/FileSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileSuite.scala
@@ -32,7 +32,6 @@ import org.apache.hadoop.mapreduce.lib.input.{FileSplit => NewFileSplit, TextInp
import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat}
import org.scalatest.FunSuite
-import org.apache.spark.SparkContext._
import org.apache.spark.rdd.{NewHadoopRDD, HadoopRDD}
import org.apache.spark.util.Utils
diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
index d27880f4bc32f..ccfe0678cb1c3 100644
--- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
@@ -120,7 +120,7 @@ class MapOutputTrackerSuite extends FunSuite {
securityManager = new SecurityManager(conf))
val slaveTracker = new MapOutputTrackerWorker(conf)
val selection = slaveSystem.actorSelection(
- s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker")
+ AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker"))
val timeout = AkkaUtils.lookupTimeout(conf)
slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout)
diff --git a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala
new file mode 100644
index 0000000000000..444a33371bd71
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark
+
+import java.io.File
+
+import com.google.common.io.Files
+import org.apache.spark.util.Utils
+import org.scalatest.{BeforeAndAfterAll, FunSuite}
+
+class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll {
+
+ test("test resolving property file as spark conf ") {
+ val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath
+ val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath
+
+ val conf = new SparkConf
+ conf.set("spark.ssl.enabled", "true")
+ conf.set("spark.ssl.keyStore", keyStorePath)
+ conf.set("spark.ssl.keyStorePassword", "password")
+ conf.set("spark.ssl.keyPassword", "password")
+ conf.set("spark.ssl.trustStore", trustStorePath)
+ conf.set("spark.ssl.trustStorePassword", "password")
+ conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA")
+ conf.set("spark.ssl.protocol", "SSLv3")
+
+ val opts = SSLOptions.parse(conf, "spark.ssl")
+
+ assert(opts.enabled === true)
+ assert(opts.trustStore.isDefined === true)
+ assert(opts.trustStore.get.getName === "truststore")
+ assert(opts.trustStore.get.getAbsolutePath === trustStorePath)
+ assert(opts.keyStore.isDefined === true)
+ assert(opts.keyStore.get.getName === "keystore")
+ assert(opts.keyStore.get.getAbsolutePath === keyStorePath)
+ assert(opts.trustStorePassword === Some("password"))
+ assert(opts.keyStorePassword === Some("password"))
+ assert(opts.keyPassword === Some("password"))
+ assert(opts.protocol === Some("SSLv3"))
+ assert(opts.enabledAlgorithms === Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA"))
+ }
+
+ test("test resolving property with defaults specified ") {
+ val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath
+ val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath
+
+ val conf = new SparkConf
+ conf.set("spark.ssl.enabled", "true")
+ conf.set("spark.ssl.keyStore", keyStorePath)
+ conf.set("spark.ssl.keyStorePassword", "password")
+ conf.set("spark.ssl.keyPassword", "password")
+ conf.set("spark.ssl.trustStore", trustStorePath)
+ conf.set("spark.ssl.trustStorePassword", "password")
+ conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA")
+ conf.set("spark.ssl.protocol", "SSLv3")
+
+ val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None)
+ val opts = SSLOptions.parse(conf, "spark.ui.ssl", defaults = Some(defaultOpts))
+
+ assert(opts.enabled === true)
+ assert(opts.trustStore.isDefined === true)
+ assert(opts.trustStore.get.getName === "truststore")
+ assert(opts.trustStore.get.getAbsolutePath === trustStorePath)
+ assert(opts.keyStore.isDefined === true)
+ assert(opts.keyStore.get.getName === "keystore")
+ assert(opts.keyStore.get.getAbsolutePath === keyStorePath)
+ assert(opts.trustStorePassword === Some("password"))
+ assert(opts.keyStorePassword === Some("password"))
+ assert(opts.keyPassword === Some("password"))
+ assert(opts.protocol === Some("SSLv3"))
+ assert(opts.enabledAlgorithms === Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA"))
+ }
+
+ test("test whether defaults can be overridden ") {
+ val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath
+ val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath
+
+ val conf = new SparkConf
+ conf.set("spark.ssl.enabled", "true")
+ conf.set("spark.ui.ssl.enabled", "false")
+ conf.set("spark.ssl.keyStore", keyStorePath)
+ conf.set("spark.ssl.keyStorePassword", "password")
+ conf.set("spark.ui.ssl.keyStorePassword", "12345")
+ conf.set("spark.ssl.keyPassword", "password")
+ conf.set("spark.ssl.trustStore", trustStorePath)
+ conf.set("spark.ssl.trustStorePassword", "password")
+ conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA")
+ conf.set("spark.ui.ssl.enabledAlgorithms", "ABC, DEF")
+ conf.set("spark.ssl.protocol", "SSLv3")
+
+ val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None)
+ val opts = SSLOptions.parse(conf, "spark.ui.ssl", defaults = Some(defaultOpts))
+
+ assert(opts.enabled === false)
+ assert(opts.trustStore.isDefined === true)
+ assert(opts.trustStore.get.getName === "truststore")
+ assert(opts.trustStore.get.getAbsolutePath === trustStorePath)
+ assert(opts.keyStore.isDefined === true)
+ assert(opts.keyStore.get.getName === "keystore")
+ assert(opts.keyStore.get.getAbsolutePath === keyStorePath)
+ assert(opts.trustStorePassword === Some("password"))
+ assert(opts.keyStorePassword === Some("12345"))
+ assert(opts.keyPassword === Some("password"))
+ assert(opts.protocol === Some("SSLv3"))
+ assert(opts.enabledAlgorithms === Set("ABC", "DEF"))
+ }
+
+}
diff --git a/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala
new file mode 100644
index 0000000000000..ace8123a8961f
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark
+
+import java.io.File
+
+object SSLSampleConfigs {
+ val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath
+ val untrustedKeyStorePath = new File(this.getClass.getResource("/untrusted-keystore").toURI).getAbsolutePath
+ val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath
+
+ def sparkSSLConfig() = {
+ val conf = new SparkConf(loadDefaults = false)
+ conf.set("spark.ssl.enabled", "true")
+ conf.set("spark.ssl.keyStore", keyStorePath)
+ conf.set("spark.ssl.keyStorePassword", "password")
+ conf.set("spark.ssl.keyPassword", "password")
+ conf.set("spark.ssl.trustStore", trustStorePath)
+ conf.set("spark.ssl.trustStorePassword", "password")
+ conf.set("spark.ssl.enabledAlgorithms",
+ "TLS_RSA_WITH_AES_128_CBC_SHA, SSL_RSA_WITH_DES_CBC_SHA")
+ conf.set("spark.ssl.protocol", "TLSv1")
+ conf
+ }
+
+ def sparkSSLConfigUntrusted() = {
+ val conf = new SparkConf(loadDefaults = false)
+ conf.set("spark.ssl.enabled", "true")
+ conf.set("spark.ssl.keyStore", untrustedKeyStorePath)
+ conf.set("spark.ssl.keyStorePassword", "password")
+ conf.set("spark.ssl.keyPassword", "password")
+ conf.set("spark.ssl.trustStore", trustStorePath)
+ conf.set("spark.ssl.trustStorePassword", "password")
+ conf.set("spark.ssl.enabledAlgorithms",
+ "TLS_RSA_WITH_AES_128_CBC_SHA, SSL_RSA_WITH_DES_CBC_SHA")
+ conf.set("spark.ssl.protocol", "TLSv1")
+ conf
+ }
+
+}
diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala
index fcca0867b8072..43fbd3ff3f756 100644
--- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala
@@ -17,7 +17,7 @@
package org.apache.spark
-import scala.collection.mutable.ArrayBuffer
+import java.io.File
import org.scalatest.FunSuite
@@ -125,6 +125,54 @@ class SecurityManagerSuite extends FunSuite {
}
+ test("ssl on setup") {
+ val conf = SSLSampleConfigs.sparkSSLConfig()
+
+ val securityManager = new SecurityManager(conf)
+
+ assert(securityManager.fileServerSSLOptions.enabled === true)
+ assert(securityManager.akkaSSLOptions.enabled === true)
+
+ assert(securityManager.sslSocketFactory.isDefined === true)
+ assert(securityManager.hostnameVerifier.isDefined === true)
+
+ assert(securityManager.fileServerSSLOptions.trustStore.isDefined === true)
+ assert(securityManager.fileServerSSLOptions.trustStore.get.getName === "truststore")
+ assert(securityManager.fileServerSSLOptions.keyStore.isDefined === true)
+ assert(securityManager.fileServerSSLOptions.keyStore.get.getName === "keystore")
+ assert(securityManager.fileServerSSLOptions.trustStorePassword === Some("password"))
+ assert(securityManager.fileServerSSLOptions.keyStorePassword === Some("password"))
+ assert(securityManager.fileServerSSLOptions.keyPassword === Some("password"))
+ assert(securityManager.fileServerSSLOptions.protocol === Some("TLSv1"))
+ assert(securityManager.fileServerSSLOptions.enabledAlgorithms ===
+ Set("TLS_RSA_WITH_AES_128_CBC_SHA", "SSL_RSA_WITH_DES_CBC_SHA"))
+
+ assert(securityManager.akkaSSLOptions.trustStore.isDefined === true)
+ assert(securityManager.akkaSSLOptions.trustStore.get.getName === "truststore")
+ assert(securityManager.akkaSSLOptions.keyStore.isDefined === true)
+ assert(securityManager.akkaSSLOptions.keyStore.get.getName === "keystore")
+ assert(securityManager.akkaSSLOptions.trustStorePassword === Some("password"))
+ assert(securityManager.akkaSSLOptions.keyStorePassword === Some("password"))
+ assert(securityManager.akkaSSLOptions.keyPassword === Some("password"))
+ assert(securityManager.akkaSSLOptions.protocol === Some("TLSv1"))
+ assert(securityManager.akkaSSLOptions.enabledAlgorithms ===
+ Set("TLS_RSA_WITH_AES_128_CBC_SHA", "SSL_RSA_WITH_DES_CBC_SHA"))
+ }
+
+ test("ssl off setup") {
+ val file = File.createTempFile("SSLOptionsSuite", "conf")
+ file.deleteOnExit()
+
+ System.setProperty("spark.ssl.configFile", file.getAbsolutePath)
+ val conf = new SparkConf()
+
+ val securityManager = new SecurityManager(conf)
+
+ assert(securityManager.fileServerSSLOptions.enabled === false)
+ assert(securityManager.akkaSSLOptions.enabled === false)
+ assert(securityManager.sslSocketFactory.isDefined === false)
+ assert(securityManager.hostnameVerifier.isDefined === false)
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
index 790976a5ac308..e08210ae60d17 100644
--- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
@@ -17,6 +17,10 @@
package org.apache.spark
+import java.util.concurrent.{TimeUnit, Executors}
+
+import scala.util.{Try, Random}
+
import org.scalatest.FunSuite
import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer}
import org.apache.spark.util.ResetSystemProperties
@@ -123,6 +127,27 @@ class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemPro
assert(conf.get("spark.test.a.b.c") === "a.b.c")
}
+ test("Thread safeness - SPARK-5425") {
+ import scala.collection.JavaConversions._
+ val executor = Executors.newSingleThreadScheduledExecutor()
+ val sf = executor.scheduleAtFixedRate(new Runnable {
+ override def run(): Unit =
+ System.setProperty("spark.5425." + Random.nextInt(), Random.nextInt().toString)
+ }, 0, 1, TimeUnit.MILLISECONDS)
+
+ try {
+ val t0 = System.currentTimeMillis()
+ while ((System.currentTimeMillis() - t0) < 1000) {
+ val conf = Try(new SparkConf(loadDefaults = true))
+ assert(conf.isSuccess === true)
+ }
+ } finally {
+ executor.shutdownNow()
+ for (key <- System.getProperties.stringPropertyNames() if key.startsWith("spark.5425."))
+ System.getProperties.remove(key)
+ }
+ }
+
test("register kryo classes through registerKryoClasses") {
val conf = new SparkConf().set("spark.kryo.registrationRequired", "true")
diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
index 8b3c6871a7b39..50f347f1954de 100644
--- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
@@ -17,10 +17,17 @@
package org.apache.spark
+import java.io.File
+
+import com.google.common.base.Charsets._
+import com.google.common.io.Files
+
import org.scalatest.FunSuite
import org.apache.hadoop.io.BytesWritable
+import org.apache.spark.util.Utils
+
class SparkContextSuite extends FunSuite with LocalSparkContext {
test("Only one SparkContext may be active at a time") {
@@ -72,4 +79,74 @@ class SparkContextSuite extends FunSuite with LocalSparkContext {
val byteArray2 = converter.convert(bytesWritable)
assert(byteArray2.length === 0)
}
+
+ test("addFile works") {
+ val file = File.createTempFile("someprefix", "somesuffix")
+ val absolutePath = file.getAbsolutePath
+ try {
+ Files.write("somewords", file, UTF_8)
+ val length = file.length()
+ sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local"))
+ sc.addFile(file.getAbsolutePath)
+ sc.parallelize(Array(1), 1).map(x => {
+ val gotten = new File(SparkFiles.get(file.getName))
+ if (!gotten.exists()) {
+ throw new SparkException("file doesn't exist")
+ }
+ if (length != gotten.length()) {
+ throw new SparkException(
+ s"file has different length $length than added file ${gotten.length()}")
+ }
+ if (absolutePath == gotten.getAbsolutePath) {
+ throw new SparkException("file should have been copied")
+ }
+ x
+ }).count()
+ } finally {
+ sc.stop()
+ }
+ }
+
+ test("addFile recursive works") {
+ val pluto = Utils.createTempDir()
+ val neptune = Utils.createTempDir(pluto.getAbsolutePath)
+ val saturn = Utils.createTempDir(neptune.getAbsolutePath)
+ val alien1 = File.createTempFile("alien", "1", neptune)
+ val alien2 = File.createTempFile("alien", "2", saturn)
+
+ try {
+ sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local"))
+ sc.addFile(neptune.getAbsolutePath, true)
+ sc.parallelize(Array(1), 1).map(x => {
+ val sep = File.separator
+ if (!new File(SparkFiles.get(neptune.getName + sep + alien1.getName)).exists()) {
+ throw new SparkException("can't access file under root added directory")
+ }
+ if (!new File(SparkFiles.get(neptune.getName + sep + saturn.getName + sep + alien2.getName))
+ .exists()) {
+ throw new SparkException("can't access file in nested directory")
+ }
+ if (new File(SparkFiles.get(pluto.getName + sep + neptune.getName + sep + alien1.getName))
+ .exists()) {
+ throw new SparkException("file exists that shouldn't")
+ }
+ x
+ }).count()
+ } finally {
+ sc.stop()
+ }
+ }
+
+ test("addFile recursive can't add directories by default") {
+ val dir = Utils.createTempDir()
+
+ try {
+ sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local"))
+ intercept[SparkException] {
+ sc.addFile(dir.getAbsolutePath)
+ }
+ } finally {
+ sc.stop()
+ }
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala
index 7b866f08a0e9f..c63d834f9048b 100644
--- a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala
@@ -23,11 +23,22 @@ import org.scalatest.FunSuite
class PythonRDDSuite extends FunSuite {
- test("Writing large strings to the worker") {
- val input: List[String] = List("a"*100000)
- val buffer = new DataOutputStream(new ByteArrayOutputStream)
- PythonRDD.writeIteratorToStream(input.iterator, buffer)
- }
+ test("Writing large strings to the worker") {
+ val input: List[String] = List("a"*100000)
+ val buffer = new DataOutputStream(new ByteArrayOutputStream)
+ PythonRDD.writeIteratorToStream(input.iterator, buffer)
+ }
+ test("Handle nulls gracefully") {
+ val buffer = new DataOutputStream(new ByteArrayOutputStream)
+ // Should not have NPE when write an Iterator with null in it
+ // The correctness will be tested in Python
+ PythonRDD.writeIteratorToStream(Iterator("a", null), buffer)
+ PythonRDD.writeIteratorToStream(Iterator(null, "a"), buffer)
+ PythonRDD.writeIteratorToStream(Iterator("a".getBytes, null), buffer)
+ PythonRDD.writeIteratorToStream(Iterator(null, "a".getBytes), buffer)
+ PythonRDD.writeIteratorToStream(Iterator((null, null), ("a", null), (null, "b")), buffer)
+ PythonRDD.writeIteratorToStream(
+ Iterator((null, null), ("a".getBytes, null), (null, "b".getBytes)), buffer)
+ }
}
-
diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala
index d2dae34be7bfb..518073dcbb64e 100644
--- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala
@@ -23,6 +23,7 @@ import org.scalatest.Matchers
class ClientSuite extends FunSuite with Matchers {
test("correctly validates driver jar URL's") {
ClientArguments.isValidJarUrl("http://someHost:8080/foo.jar") should be (true)
+ ClientArguments.isValidJarUrl("https://someHost:8080/foo.jar") should be (true)
// file scheme with authority and path is valid.
ClientArguments.isValidJarUrl("file://somehost/path/to/a/jarFile.jar") should be (true)
diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
index aa65f7e8915e6..68b5776fc6515 100644
--- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
@@ -68,7 +68,8 @@ class JsonProtocolSuite extends FunSuite {
val completedApps = Array[ApplicationInfo]()
val activeDrivers = Array(createDriverInfo())
val completedDrivers = Array(createDriverInfo())
- val stateResponse = new MasterStateResponse("host", 8080, workers, activeApps, completedApps,
+ val stateResponse = new MasterStateResponse(
+ "host", 8080, None, workers, activeApps, completedApps,
activeDrivers, completedDrivers, RecoveryState.ALIVE)
val output = JsonProtocol.writeMasterState(stateResponse)
assertValidJson(output)
@@ -117,8 +118,8 @@ class JsonProtocolSuite extends FunSuite {
}
def createExecutorRunner(): ExecutorRunner = {
- new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host",
- new File("sparkHome"), new File("workDir"), "akka://worker",
+ new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", 123,
+ "publicAddress", new File("sparkHome"), new File("workDir"), "akka://worker",
new SparkConf, Seq("localDir"), ExecutorState.RUNNING)
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala
new file mode 100644
index 0000000000000..54dd7c9c45c61
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala
@@ -0,0 +1,93 @@
+/*
+ * 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
+
+import java.net.URL
+
+import scala.collection.mutable
+import scala.io.Source
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.scheduler.cluster.ExecutorInfo
+import org.apache.spark.scheduler.{SparkListenerExecutorAdded, SparkListener}
+import org.apache.spark.{SparkConf, SparkContext, LocalSparkContext}
+
+class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext {
+
+ /** Length of time to wait while draining listener events. */
+ private val WAIT_TIMEOUT_MILLIS = 10000
+
+ test("verify that correct log urls get propagated from workers") {
+ sc = new SparkContext("local-cluster[2,1,512]", "test")
+
+ val listener = new SaveExecutorInfo
+ sc.addSparkListener(listener)
+
+ // Trigger a job so that executors get added
+ sc.parallelize(1 to 100, 4).map(_.toString).count()
+
+ assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
+ listener.addedExecutorInfos.values.foreach { info =>
+ assert(info.logUrlMap.nonEmpty)
+ // Browse to each URL to check that it's valid
+ info.logUrlMap.foreach { case (logType, logUrl) =>
+ val html = Source.fromURL(logUrl).mkString
+ assert(html.contains(s"$logType log page"))
+ }
+ }
+ }
+
+ test("verify that log urls reflect SPARK_PUBLIC_DNS (SPARK-6175)") {
+ val SPARK_PUBLIC_DNS = "public_dns"
+ class MySparkConf extends SparkConf(false) {
+ override def getenv(name: String) = {
+ if (name == "SPARK_PUBLIC_DNS") SPARK_PUBLIC_DNS
+ else super.getenv(name)
+ }
+
+ override def clone: SparkConf = {
+ new MySparkConf().setAll(getAll)
+ }
+ }
+ val conf = new MySparkConf()
+ sc = new SparkContext("local-cluster[2,1,512]", "test", conf)
+
+ val listener = new SaveExecutorInfo
+ sc.addSparkListener(listener)
+
+ // Trigger a job so that executors get added
+ sc.parallelize(1 to 100, 4).map(_.toString).count()
+
+ assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
+ listener.addedExecutorInfos.values.foreach { info =>
+ assert(info.logUrlMap.nonEmpty)
+ info.logUrlMap.values.foreach { logUrl =>
+ assert(new URL(logUrl).getHost === SPARK_PUBLIC_DNS)
+ }
+ }
+ }
+
+ private class SaveExecutorInfo extends SparkListener {
+ val addedExecutorInfos = mutable.Map[String, ExecutorInfo]()
+
+ override def onExecutorAdded(executor: SparkListenerExecutorAdded) {
+ addedExecutorInfos(executor.executorId) = executor.executorInfo
+ }
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
index 82628ad3abd99..46d745c4ecbfa 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -21,6 +21,8 @@ import java.io._
import scala.collection.mutable.ArrayBuffer
+import com.google.common.base.Charsets.UTF_8
+import com.google.common.io.ByteStreams
import org.scalatest.FunSuite
import org.scalatest.Matchers
import org.scalatest.concurrent.Timeouts
@@ -141,7 +143,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
"thejar.jar",
"arg1", "arg2")
val appArgs = new SparkSubmitArguments(clArgs)
- val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs)
+ val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs)
val childArgsStr = childArgs.mkString(" ")
childArgsStr should include ("--class org.SomeClass")
childArgsStr should include ("--executor-memory 5g")
@@ -180,7 +182,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
"thejar.jar",
"arg1", "arg2")
val appArgs = new SparkSubmitArguments(clArgs)
- val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs)
+ val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs)
childArgs.mkString(" ") should be ("arg1 arg2")
mainClass should be ("org.SomeClass")
classpath should have length (4)
@@ -201,6 +203,18 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
}
test("handles standalone cluster mode") {
+ testStandaloneCluster(useRest = true)
+ }
+
+ test("handles legacy standalone cluster mode") {
+ testStandaloneCluster(useRest = false)
+ }
+
+ /**
+ * Test whether the launch environment is correctly set up in standalone cluster mode.
+ * @param useRest whether to use the REST submission gateway introduced in Spark 1.3
+ */
+ private def testStandaloneCluster(useRest: Boolean): Unit = {
val clArgs = Seq(
"--deploy-mode", "cluster",
"--master", "spark://h:p",
@@ -212,17 +226,26 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
"thejar.jar",
"arg1", "arg2")
val appArgs = new SparkSubmitArguments(clArgs)
- val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs)
+ appArgs.useRest = useRest
+ val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs)
val childArgsStr = childArgs.mkString(" ")
- childArgsStr should startWith ("--memory 4g --cores 5 --supervise")
- childArgsStr should include regex ("launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2")
- mainClass should be ("org.apache.spark.deploy.Client")
- classpath should have size (0)
- sysProps should have size (5)
+ if (useRest) {
+ childArgsStr should endWith ("thejar.jar org.SomeClass arg1 arg2")
+ mainClass should be ("org.apache.spark.deploy.rest.StandaloneRestClient")
+ } else {
+ childArgsStr should startWith ("--supervise --memory 4g --cores 5")
+ childArgsStr should include regex "launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2"
+ mainClass should be ("org.apache.spark.deploy.Client")
+ }
+ classpath should have size 0
+ sysProps should have size 8
sysProps.keys should contain ("SPARK_SUBMIT")
sysProps.keys should contain ("spark.master")
sysProps.keys should contain ("spark.app.name")
sysProps.keys should contain ("spark.jars")
+ sysProps.keys should contain ("spark.driver.memory")
+ sysProps.keys should contain ("spark.driver.cores")
+ sysProps.keys should contain ("spark.driver.supervise")
sysProps.keys should contain ("spark.shuffle.spill")
sysProps("spark.shuffle.spill") should be ("false")
}
@@ -239,7 +262,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
"thejar.jar",
"arg1", "arg2")
val appArgs = new SparkSubmitArguments(clArgs)
- val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs)
+ val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs)
childArgs.mkString(" ") should be ("arg1 arg2")
mainClass should be ("org.SomeClass")
classpath should have length (1)
@@ -261,7 +284,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
"thejar.jar",
"arg1", "arg2")
val appArgs = new SparkSubmitArguments(clArgs)
- val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs)
+ val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs)
childArgs.mkString(" ") should be ("arg1 arg2")
mainClass should be ("org.SomeClass")
classpath should have length (1)
@@ -281,7 +304,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
"thejar.jar",
"arg1", "arg2")
val appArgs = new SparkSubmitArguments(clArgs)
- val (_, _, sysProps, mainClass) = createLaunchEnv(appArgs)
+ val (_, _, sysProps, mainClass) = prepareSubmitEnvironment(appArgs)
sysProps("spark.executor.memory") should be ("5g")
sysProps("spark.master") should be ("yarn-cluster")
mainClass should be ("org.apache.spark.deploy.yarn.Client")
@@ -307,7 +330,21 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
"--name", "testApp",
"--master", "local-cluster[2,1,512]",
"--jars", jarsString,
- unusedJar.toString)
+ unusedJar.toString, "SparkSubmitClassA", "SparkSubmitClassB")
+ runSparkSubmit(args)
+ }
+
+ test("includes jars passed in through --packages") {
+ val unusedJar = TestUtils.createJarWithClasses(Seq.empty)
+ val packagesString = "com.databricks:spark-csv_2.10:0.1,com.databricks:spark-avro_2.10:0.1"
+ val args = Seq(
+ "--class", JarCreationTest.getClass.getName.stripSuffix("$"),
+ "--name", "testApp",
+ "--master", "local-cluster[2,1,512]",
+ "--packages", packagesString,
+ "--conf", "spark.ui.enabled=false",
+ unusedJar.toString,
+ "com.databricks.spark.csv.DefaultSource", "com.databricks.spark.avro.DefaultSource")
runSparkSubmit(args)
}
@@ -325,7 +362,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
"--files", files,
"thejar.jar")
val appArgs = new SparkSubmitArguments(clArgs)
- val sysProps = SparkSubmit.createLaunchEnv(appArgs)._3
+ val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs)._3
appArgs.jars should be (Utils.resolveURIs(jars))
appArgs.files should be (Utils.resolveURIs(files))
sysProps("spark.jars") should be (Utils.resolveURIs(jars + ",thejar.jar"))
@@ -340,7 +377,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
"thejar.jar"
)
val appArgs2 = new SparkSubmitArguments(clArgs2)
- val sysProps2 = SparkSubmit.createLaunchEnv(appArgs2)._3
+ val sysProps2 = SparkSubmit.prepareSubmitEnvironment(appArgs2)._3
appArgs2.files should be (Utils.resolveURIs(files))
appArgs2.archives should be (Utils.resolveURIs(archives))
sysProps2("spark.yarn.dist.files") should be (Utils.resolveURIs(files))
@@ -353,7 +390,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
"mister.py"
)
val appArgs3 = new SparkSubmitArguments(clArgs3)
- val sysProps3 = SparkSubmit.createLaunchEnv(appArgs3)._3
+ val sysProps3 = SparkSubmit.prepareSubmitEnvironment(appArgs3)._3
appArgs3.pyFiles should be (Utils.resolveURIs(pyFiles))
sysProps3("spark.submit.pyFiles") should be (
PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(","))
@@ -378,7 +415,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
"thejar.jar"
)
val appArgs = new SparkSubmitArguments(clArgs)
- val sysProps = SparkSubmit.createLaunchEnv(appArgs)._3
+ val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs)._3
sysProps("spark.jars") should be(Utils.resolveURIs(jars + ",thejar.jar"))
sysProps("spark.files") should be(Utils.resolveURIs(files))
@@ -395,7 +432,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
"thejar.jar"
)
val appArgs2 = new SparkSubmitArguments(clArgs2)
- val sysProps2 = SparkSubmit.createLaunchEnv(appArgs2)._3
+ val sysProps2 = SparkSubmit.prepareSubmitEnvironment(appArgs2)._3
sysProps2("spark.yarn.dist.files") should be(Utils.resolveURIs(files))
sysProps2("spark.yarn.dist.archives") should be(Utils.resolveURIs(archives))
@@ -410,11 +447,24 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
"mister.py"
)
val appArgs3 = new SparkSubmitArguments(clArgs3)
- val sysProps3 = SparkSubmit.createLaunchEnv(appArgs3)._3
+ val sysProps3 = SparkSubmit.prepareSubmitEnvironment(appArgs3)._3
sysProps3("spark.submit.pyFiles") should be(
PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(","))
}
+ test("user classpath first in driver") {
+ val systemJar = TestUtils.createJarWithFiles(Map("test.resource" -> "SYSTEM"))
+ val userJar = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"))
+ val args = Seq(
+ "--class", UserClasspathFirstTest.getClass.getName.stripSuffix("$"),
+ "--name", "testApp",
+ "--master", "local",
+ "--conf", "spark.driver.extraClassPath=" + systemJar,
+ "--conf", "spark.driver.userClassPathFirst=true",
+ userJar.toString)
+ runSparkSubmit(args)
+ }
+
test("SPARK_CONF_DIR overrides spark-defaults.conf") {
forConfDir(Map("spark.executor.memory" -> "2.3g")) { path =>
val unusedJar = TestUtils.createJarWithClasses(Seq.empty)
@@ -426,7 +476,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties
val appArgs = new SparkSubmitArguments(args, Map("SPARK_CONF_DIR" -> path))
assert(appArgs.propertiesFile != null)
assert(appArgs.propertiesFile.startsWith(path))
- appArgs.executorMemory should be ("2.3g")
+ appArgs.executorMemory should be ("2.3g")
}
}
@@ -467,8 +517,8 @@ object JarCreationTest extends Logging {
val result = sc.makeRDD(1 to 100, 10).mapPartitions { x =>
var exception: String = null
try {
- Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader)
- Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader)
+ Class.forName(args(0), true, Thread.currentThread().getContextClassLoader)
+ Class.forName(args(1), true, Thread.currentThread().getContextClassLoader)
} catch {
case t: Throwable =>
exception = t + "\n" + t.getStackTraceString
@@ -506,3 +556,15 @@ object SimpleApplicationTest {
}
}
}
+
+object UserClasspathFirstTest {
+ def main(args: Array[String]) {
+ val ccl = Thread.currentThread().getContextClassLoader()
+ val resource = ccl.getResourceAsStream("test.resource")
+ val bytes = ByteStreams.toByteArray(resource)
+ val contents = new String(bytes, 0, bytes.length, UTF_8)
+ if (contents != "USER") {
+ throw new SparkException("Should have read user resource, but instead read: " + contents)
+ }
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala
new file mode 100644
index 0000000000000..8bcca926097a1
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala
@@ -0,0 +1,136 @@
+/*
+ * 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
+
+import java.io.{PrintStream, OutputStream, File}
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.scalatest.{BeforeAndAfterAll, FunSuite}
+
+import org.apache.ivy.core.module.descriptor.MDArtifact
+import org.apache.ivy.plugins.resolver.IBiblioResolver
+
+class SparkSubmitUtilsSuite extends FunSuite with BeforeAndAfterAll {
+
+ private val noOpOutputStream = new OutputStream {
+ def write(b: Int) = {}
+ }
+
+ /** Simple PrintStream that reads data into a buffer */
+ private class BufferPrintStream extends PrintStream(noOpOutputStream) {
+ var lineBuffer = ArrayBuffer[String]()
+ override def println(line: String) {
+ lineBuffer += line
+ }
+ }
+
+ override def beforeAll() {
+ super.beforeAll()
+ // We don't want to write logs during testing
+ SparkSubmitUtils.printStream = new BufferPrintStream
+ }
+
+ test("incorrect maven coordinate throws error") {
+ val coordinates = Seq("a:b: ", " :a:b", "a: :b", "a:b:", ":a:b", "a::b", "::", "a:b", "a")
+ for (coordinate <- coordinates) {
+ intercept[IllegalArgumentException] {
+ SparkSubmitUtils.extractMavenCoordinates(coordinate)
+ }
+ }
+ }
+
+ test("create repo resolvers") {
+ val resolver1 = SparkSubmitUtils.createRepoResolvers(None)
+ // should have central and spark-packages by default
+ assert(resolver1.getResolvers.size() === 2)
+ assert(resolver1.getResolvers.get(0).asInstanceOf[IBiblioResolver].getName === "central")
+ assert(resolver1.getResolvers.get(1).asInstanceOf[IBiblioResolver].getName === "spark-packages")
+
+ val repos = "a/1,b/2,c/3"
+ val resolver2 = SparkSubmitUtils.createRepoResolvers(Option(repos))
+ assert(resolver2.getResolvers.size() === 5)
+ val expected = repos.split(",").map(r => s"$r/")
+ resolver2.getResolvers.toArray.zipWithIndex.foreach { case (resolver: IBiblioResolver, i) =>
+ if (i == 0) {
+ assert(resolver.getName === "central")
+ } else if (i == 1) {
+ assert(resolver.getName === "spark-packages")
+ } else {
+ assert(resolver.getName === s"repo-${i - 1}")
+ assert(resolver.getRoot === expected(i - 2))
+ }
+ }
+ }
+
+ test("add dependencies works correctly") {
+ val md = SparkSubmitUtils.getModuleDescriptor
+ val artifacts = SparkSubmitUtils.extractMavenCoordinates("com.databricks:spark-csv_2.10:0.1," +
+ "com.databricks:spark-avro_2.10:0.1")
+
+ SparkSubmitUtils.addDependenciesToIvy(md, artifacts, "default")
+ assert(md.getDependencies.length === 2)
+ }
+
+ test("ivy path works correctly") {
+ val ivyPath = "dummy/ivy"
+ val md = SparkSubmitUtils.getModuleDescriptor
+ val artifacts = for (i <- 0 until 3) yield new MDArtifact(md, s"jar-$i", "jar", "jar")
+ var jPaths = SparkSubmitUtils.resolveDependencyPaths(artifacts.toArray, new File(ivyPath))
+ for (i <- 0 until 3) {
+ val index = jPaths.indexOf(ivyPath)
+ assert(index >= 0)
+ jPaths = jPaths.substring(index + ivyPath.length)
+ }
+ // end to end
+ val jarPath = SparkSubmitUtils.resolveMavenCoordinates(
+ "com.databricks:spark-csv_2.10:0.1", None, Option(ivyPath), true)
+ assert(jarPath.indexOf(ivyPath) >= 0, "should use non-default ivy path")
+ }
+
+ test("search for artifact at other repositories") {
+ val path = SparkSubmitUtils.resolveMavenCoordinates("com.agimatec:agimatec-validation:0.9.3",
+ Option("https://oss.sonatype.org/content/repositories/agimatec/"), None, true)
+ assert(path.indexOf("agimatec-validation") >= 0, "should find package. If it doesn't, check" +
+ "if package still exists. If it has been removed, replace the example in this test.")
+ }
+
+ test("dependency not found throws RuntimeException") {
+ intercept[RuntimeException] {
+ SparkSubmitUtils.resolveMavenCoordinates("a:b:c", None, None, true)
+ }
+ }
+
+ test("neglects Spark and Spark's dependencies") {
+ val components = Seq("bagel_", "catalyst_", "core_", "graphx_", "hive_", "mllib_", "repl_",
+ "sql_", "streaming_", "yarn_", "network-common_", "network-shuffle_", "network-yarn_")
+
+ val coordinates =
+ components.map(comp => s"org.apache.spark:spark-${comp}2.10:1.2.0").mkString(",") +
+ ",org.apache.spark:spark-core_fake:1.2.0"
+
+ val path = SparkSubmitUtils.resolveMavenCoordinates(coordinates, None, None, true)
+ assert(path === "", "should return empty path")
+ // Should not exclude the following dependency. Will throw an error, because it doesn't exist,
+ // but the fact that it is checking means that it wasn't excluded.
+ intercept[RuntimeException] {
+ SparkSubmitUtils.resolveMavenCoordinates(coordinates +
+ ",org.apache.spark:spark-streaming-kafka-assembly_2.10:1.2.0", None, None, true)
+ }
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
index 3fbc1a21d10ed..e908ba604ebed 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
@@ -17,18 +17,17 @@
package org.apache.spark.deploy.history
-import java.io.{File, FileOutputStream, OutputStreamWriter}
+import java.io.{BufferedOutputStream, File, FileOutputStream, OutputStreamWriter}
+import java.net.URI
import scala.io.Source
-import com.google.common.io.Files
import org.apache.hadoop.fs.Path
import org.json4s.jackson.JsonMethods._
import org.scalatest.{BeforeAndAfter, FunSuite}
import org.scalatest.Matchers
import org.apache.spark.{Logging, SparkConf}
-import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.io._
import org.apache.spark.scheduler._
import org.apache.spark.util.{JsonProtocol, Utils}
@@ -37,54 +36,67 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers
private var testDir: File = null
- private var provider: FsHistoryProvider = null
-
before {
testDir = Utils.createTempDir()
- provider = new FsHistoryProvider(new SparkConf()
- .set("spark.history.fs.logDirectory", testDir.getAbsolutePath())
- .set("spark.history.fs.updateInterval", "0"))
}
after {
Utils.deleteRecursively(testDir)
}
+ /** Create a fake log file using the new log format used in Spark 1.3+ */
+ private def newLogFile(
+ appId: String,
+ inProgress: Boolean,
+ codec: Option[String] = None): File = {
+ val ip = if (inProgress) EventLoggingListener.IN_PROGRESS else ""
+ val logUri = EventLoggingListener.getLogPath(testDir.getAbsolutePath, appId)
+ val logPath = new URI(logUri).getPath + ip
+ new File(logPath)
+ }
+
test("Parse new and old application logs") {
- val conf = new SparkConf()
- .set("spark.history.fs.logDirectory", testDir.getAbsolutePath())
- .set("spark.history.fs.updateInterval", "0")
- val provider = new FsHistoryProvider(conf)
+ val provider = new FsHistoryProvider(createTestConf())
// Write a new-style application log.
- val logFile1 = new File(testDir, "new1")
- writeFile(logFile1, true, None,
- SparkListenerApplicationStart("app1-1", None, 1L, "test"),
- SparkListenerApplicationEnd(2L)
+ val newAppComplete = newLogFile("new1", inProgress = false)
+ writeFile(newAppComplete, true, None,
+ SparkListenerApplicationStart("new-app-complete", None, 1L, "test"),
+ SparkListenerApplicationEnd(5L)
)
+ // Write a new-style application log.
+ val newAppCompressedComplete = newLogFile("new1compressed", inProgress = false, Some("lzf"))
+ writeFile(newAppCompressedComplete, true, None,
+ SparkListenerApplicationStart("new-app-compressed-complete", None, 1L, "test"),
+ SparkListenerApplicationEnd(4L))
+
// Write an unfinished app, new-style.
- val logFile2 = new File(testDir, "new2" + EventLoggingListener.IN_PROGRESS)
- writeFile(logFile2, true, None,
- SparkListenerApplicationStart("app2-2", None, 1L, "test")
+ val newAppIncomplete = newLogFile("new2", inProgress = true)
+ writeFile(newAppIncomplete, true, None,
+ SparkListenerApplicationStart("new-app-incomplete", None, 1L, "test")
)
// Write an old-style application log.
- val oldLog = new File(testDir, "old1")
- oldLog.mkdir()
- createEmptyFile(new File(oldLog, provider.SPARK_VERSION_PREFIX + "1.0"))
- writeFile(new File(oldLog, provider.LOG_PREFIX + "1"), false, None,
- SparkListenerApplicationStart("app3", None, 2L, "test"),
+ val oldAppComplete = new File(testDir, "old1")
+ oldAppComplete.mkdir()
+ createEmptyFile(new File(oldAppComplete, provider.SPARK_VERSION_PREFIX + "1.0"))
+ writeFile(new File(oldAppComplete, provider.LOG_PREFIX + "1"), false, None,
+ SparkListenerApplicationStart("old-app-complete", None, 2L, "test"),
SparkListenerApplicationEnd(3L)
)
- createEmptyFile(new File(oldLog, provider.APPLICATION_COMPLETE))
+ createEmptyFile(new File(oldAppComplete, provider.APPLICATION_COMPLETE))
+
+ // Check for logs so that we force the older unfinished app to be loaded, to make
+ // sure unfinished apps are also sorted correctly.
+ provider.checkForLogs()
// Write an unfinished app, old-style.
- val oldLog2 = new File(testDir, "old2")
- oldLog2.mkdir()
- createEmptyFile(new File(oldLog2, provider.SPARK_VERSION_PREFIX + "1.0"))
- writeFile(new File(oldLog2, provider.LOG_PREFIX + "1"), false, None,
- SparkListenerApplicationStart("app4", None, 2L, "test")
+ val oldAppIncomplete = new File(testDir, "old2")
+ oldAppIncomplete.mkdir()
+ createEmptyFile(new File(oldAppIncomplete, provider.SPARK_VERSION_PREFIX + "1.0"))
+ writeFile(new File(oldAppIncomplete, provider.LOG_PREFIX + "1"), false, None,
+ SparkListenerApplicationStart("old-app-incomplete", None, 2L, "test")
)
// Force a reload of data from the log directory, and check that both logs are loaded.
@@ -93,17 +105,19 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers
val list = provider.getListing().toSeq
list should not be (null)
- list.size should be (4)
- list.count(e => e.completed) should be (2)
-
- list(0) should be (ApplicationHistoryInfo(oldLog.getName(), "app3", 2L, 3L,
- oldLog.lastModified(), "test", true))
- list(1) should be (ApplicationHistoryInfo(logFile1.getName(), "app1-1", 1L, 2L,
- logFile1.lastModified(), "test", true))
- list(2) should be (ApplicationHistoryInfo(oldLog2.getName(), "app4", 2L, -1L,
- oldLog2.lastModified(), "test", false))
- list(3) should be (ApplicationHistoryInfo(logFile2.getName(), "app2-2", 1L, -1L,
- logFile2.lastModified(), "test", false))
+ list.size should be (5)
+ list.count(_.completed) should be (3)
+
+ list(0) should be (ApplicationHistoryInfo(newAppComplete.getName(), "new-app-complete", 1L, 5L,
+ newAppComplete.lastModified(), "test", true))
+ list(1) should be (ApplicationHistoryInfo(newAppCompressedComplete.getName(),
+ "new-app-compressed-complete", 1L, 4L, newAppCompressedComplete.lastModified(), "test", true))
+ list(2) should be (ApplicationHistoryInfo(oldAppComplete.getName(), "old-app-complete", 2L, 3L,
+ oldAppComplete.lastModified(), "test", true))
+ list(3) should be (ApplicationHistoryInfo(oldAppIncomplete.getName(), "old-app-incomplete", 2L,
+ -1L, oldAppIncomplete.lastModified(), "test", false))
+ list(4) should be (ApplicationHistoryInfo(newAppIncomplete.getName(), "new-app-incomplete", 1L,
+ -1L, newAppIncomplete.lastModified(), "test", false))
// Make sure the UI can be rendered.
list.foreach { case info =>
@@ -113,6 +127,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers
}
test("Parse legacy logs with compression codec set") {
+ val provider = new FsHistoryProvider(createTestConf())
val testCodecs = List((classOf[LZFCompressionCodec].getName(), true),
(classOf[SnappyCompressionCodec].getName(), true),
("invalid.codec", false))
@@ -130,7 +145,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers
val logPath = new Path(logDir.getAbsolutePath())
try {
- val (logInput, sparkVersion) = provider.openLegacyEventLog(logPath)
+ val logInput = provider.openLegacyEventLog(logPath)
try {
Source.fromInputStream(logInput).getLines().toSeq.size should be (2)
} finally {
@@ -144,22 +159,19 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers
}
test("SPARK-3697: ignore directories that cannot be read.") {
- val logFile1 = new File(testDir, "new1")
+ val logFile1 = newLogFile("new1", inProgress = false)
writeFile(logFile1, true, None,
SparkListenerApplicationStart("app1-1", None, 1L, "test"),
SparkListenerApplicationEnd(2L)
)
- val logFile2 = new File(testDir, "new2")
+ val logFile2 = newLogFile("new2", inProgress = false)
writeFile(logFile2, true, None,
SparkListenerApplicationStart("app1-2", None, 1L, "test"),
SparkListenerApplicationEnd(2L)
)
logFile2.setReadable(false, false)
- val conf = new SparkConf()
- .set("spark.history.fs.logDirectory", testDir.getAbsolutePath())
- .set("spark.history.fs.updateInterval", "0")
- val provider = new FsHistoryProvider(conf)
+ val provider = new FsHistoryProvider(createTestConf())
provider.checkForLogs()
val list = provider.getListing().toSeq
@@ -168,12 +180,9 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers
}
test("history file is renamed from inprogress to completed") {
- val conf = new SparkConf()
- .set("spark.history.fs.logDirectory", testDir.getAbsolutePath())
- .set("spark.testing", "true")
- val provider = new FsHistoryProvider(conf)
+ val provider = new FsHistoryProvider(createTestConf())
- val logFile1 = new File(testDir, "app1" + EventLoggingListener.IN_PROGRESS)
+ val logFile1 = newLogFile("app1", inProgress = true)
writeFile(logFile1, true, None,
SparkListenerApplicationStart("app1", Some("app1"), 1L, "test"),
SparkListenerApplicationEnd(2L)
@@ -183,23 +192,38 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers
appListBeforeRename.size should be (1)
appListBeforeRename.head.logPath should endWith(EventLoggingListener.IN_PROGRESS)
- logFile1.renameTo(new File(testDir, "app1"))
+ logFile1.renameTo(newLogFile("app1", inProgress = false))
provider.checkForLogs()
val appListAfterRename = provider.getListing()
appListAfterRename.size should be (1)
appListAfterRename.head.logPath should not endWith(EventLoggingListener.IN_PROGRESS)
}
+ test("SPARK-5582: empty log directory") {
+ val provider = new FsHistoryProvider(createTestConf())
+
+ val logFile1 = newLogFile("app1", inProgress = true)
+ writeFile(logFile1, true, None,
+ SparkListenerApplicationStart("app1", Some("app1"), 1L, "test"),
+ SparkListenerApplicationEnd(2L))
+
+ val oldLog = new File(testDir, "old1")
+ oldLog.mkdir()
+
+ provider.checkForLogs()
+ val appListAfterRename = provider.getListing()
+ appListAfterRename.size should be (1)
+ }
+
private def writeFile(file: File, isNewFormat: Boolean, codec: Option[CompressionCodec],
events: SparkListenerEvent*) = {
- val out =
- if (isNewFormat) {
- EventLoggingListener.initEventLog(new FileOutputStream(file), codec)
- } else {
- val fileStream = new FileOutputStream(file)
- codec.map(_.compressedOutputStream(fileStream)).getOrElse(fileStream)
- }
- val writer = new OutputStreamWriter(out, "UTF-8")
+ val fstream = new FileOutputStream(file)
+ val cstream = codec.map(_.compressedOutputStream(fstream)).getOrElse(fstream)
+ val bstream = new BufferedOutputStream(cstream)
+ if (isNewFormat) {
+ EventLoggingListener.initEventLog(new FileOutputStream(file))
+ }
+ val writer = new OutputStreamWriter(bstream, "UTF-8")
try {
events.foreach(e => writer.write(compact(render(JsonProtocol.sparkEventToJson(e))) + "\n"))
} finally {
@@ -211,4 +235,8 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers
new FileOutputStream(file).close()
}
+ private def createTestConf(): SparkConf = {
+ new SparkConf().set("spark.history.fs.logDirectory", testDir.getAbsolutePath())
+ }
+
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
new file mode 100644
index 0000000000000..3a9963a5ce7b7
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
@@ -0,0 +1,56 @@
+/*
+ * 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 javax.servlet.http.HttpServletRequest
+
+import scala.collection.mutable
+
+import org.apache.hadoop.fs.Path
+import org.mockito.Mockito.{when}
+import org.scalatest.FunSuite
+import org.scalatest.Matchers
+import org.scalatest.mock.MockitoSugar
+
+import org.apache.spark.ui.SparkUI
+
+class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar {
+
+ test("generate history page with relative links") {
+ val historyServer = mock[HistoryServer]
+ val request = mock[HttpServletRequest]
+ val ui = mock[SparkUI]
+ val link = "/history/app1"
+ val info = new ApplicationHistoryInfo("app1", "app1", 0, 2, 1, "xxx", true)
+ when(historyServer.getApplicationList()).thenReturn(Seq(info))
+ when(ui.basePath).thenReturn(link)
+ when(historyServer.getProviderConfig()).thenReturn(Map[String, String]())
+ val page = new HistoryPage(historyServer)
+
+ //when
+ val response = page.render(request)
+
+ //then
+ val links = response \\ "a"
+ val justHrefs = for {
+ l <- links
+ attrs <- l.attribute("href")
+ } yield (attrs.toString)
+ justHrefs should contain(link)
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
index 3d2335f9b3637..34c74d87f0a62 100644
--- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
@@ -20,30 +20,46 @@ package org.apache.spark.deploy.master
import akka.actor.Address
import org.scalatest.FunSuite
-import org.apache.spark.SparkException
+import org.apache.spark.{SSLOptions, SparkConf, SparkException}
class MasterSuite extends FunSuite {
test("toAkkaUrl") {
- val akkaUrl = Master.toAkkaUrl("spark://1.2.3.4:1234")
+ val conf = new SparkConf(loadDefaults = false)
+ val akkaUrl = Master.toAkkaUrl("spark://1.2.3.4:1234", "akka.tcp")
assert("akka.tcp://sparkMaster@1.2.3.4:1234/user/Master" === akkaUrl)
}
+ test("toAkkaUrl with SSL") {
+ val conf = new SparkConf(loadDefaults = false)
+ val akkaUrl = Master.toAkkaUrl("spark://1.2.3.4:1234", "akka.ssl.tcp")
+ assert("akka.ssl.tcp://sparkMaster@1.2.3.4:1234/user/Master" === akkaUrl)
+ }
+
test("toAkkaUrl: a typo url") {
+ val conf = new SparkConf(loadDefaults = false)
val e = intercept[SparkException] {
- Master.toAkkaUrl("spark://1.2. 3.4:1234")
+ Master.toAkkaUrl("spark://1.2. 3.4:1234", "akka.tcp")
}
assert("Invalid master URL: spark://1.2. 3.4:1234" === e.getMessage)
}
test("toAkkaAddress") {
- val address = Master.toAkkaAddress("spark://1.2.3.4:1234")
+ val conf = new SparkConf(loadDefaults = false)
+ val address = Master.toAkkaAddress("spark://1.2.3.4:1234", "akka.tcp")
assert(Address("akka.tcp", "sparkMaster", "1.2.3.4", 1234) === address)
}
+ test("toAkkaAddress with SSL") {
+ val conf = new SparkConf(loadDefaults = false)
+ val address = Master.toAkkaAddress("spark://1.2.3.4:1234", "akka.ssl.tcp")
+ assert(Address("akka.ssl.tcp", "sparkMaster", "1.2.3.4", 1234) === address)
+ }
+
test("toAkkaAddress: a typo url") {
+ val conf = new SparkConf(loadDefaults = false)
val e = intercept[SparkException] {
- Master.toAkkaAddress("spark://1.2. 3.4:1234")
+ Master.toAkkaAddress("spark://1.2. 3.4:1234", "akka.tcp")
}
assert("Invalid master URL: spark://1.2. 3.4:1234" === e.getMessage)
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala
new file mode 100644
index 0000000000000..2fa90e3bd1c63
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala
@@ -0,0 +1,606 @@
+/*
+ * 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.rest
+
+import java.io.DataOutputStream
+import java.net.{HttpURLConnection, URL}
+import javax.servlet.http.HttpServletResponse
+
+import scala.collection.mutable
+
+import akka.actor.{Actor, ActorRef, ActorSystem, Props}
+import com.google.common.base.Charsets
+import org.scalatest.{BeforeAndAfterEach, FunSuite}
+import org.json4s.JsonAST._
+import org.json4s.jackson.JsonMethods._
+
+import org.apache.spark._
+import org.apache.spark.util.{AkkaUtils, Utils}
+import org.apache.spark.deploy.DeployMessages._
+import org.apache.spark.deploy.{SparkSubmit, SparkSubmitArguments}
+import org.apache.spark.deploy.master.DriverState._
+
+/**
+ * Tests for the REST application submission protocol used in standalone cluster mode.
+ */
+class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach {
+ private val client = new StandaloneRestClient
+ private var actorSystem: Option[ActorSystem] = None
+ private var server: Option[StandaloneRestServer] = None
+
+ override def afterEach() {
+ actorSystem.foreach(_.shutdown())
+ server.foreach(_.stop())
+ }
+
+ test("construct submit request") {
+ val appArgs = Array("one", "two", "three")
+ val sparkProperties = Map("spark.app.name" -> "pi")
+ val environmentVariables = Map("SPARK_ONE" -> "UN", "SPARK_TWO" -> "DEUX")
+ val request = client.constructSubmitRequest(
+ "my-app-resource", "my-main-class", appArgs, sparkProperties, environmentVariables)
+ assert(request.action === Utils.getFormattedClassName(request))
+ assert(request.clientSparkVersion === SPARK_VERSION)
+ assert(request.appResource === "my-app-resource")
+ assert(request.mainClass === "my-main-class")
+ assert(request.appArgs === appArgs)
+ assert(request.sparkProperties === sparkProperties)
+ assert(request.environmentVariables === environmentVariables)
+ }
+
+ test("create submission") {
+ val submittedDriverId = "my-driver-id"
+ val submitMessage = "your driver is submitted"
+ val masterUrl = startDummyServer(submitId = submittedDriverId, submitMessage = submitMessage)
+ val appArgs = Array("one", "two", "four")
+ val request = constructSubmitRequest(masterUrl, appArgs)
+ assert(request.appArgs === appArgs)
+ assert(request.sparkProperties("spark.master") === masterUrl)
+ val response = client.createSubmission(masterUrl, request)
+ val submitResponse = getSubmitResponse(response)
+ assert(submitResponse.action === Utils.getFormattedClassName(submitResponse))
+ assert(submitResponse.serverSparkVersion === SPARK_VERSION)
+ assert(submitResponse.message === submitMessage)
+ assert(submitResponse.submissionId === submittedDriverId)
+ assert(submitResponse.success)
+ }
+
+ test("create submission from main method") {
+ val submittedDriverId = "your-driver-id"
+ val submitMessage = "my driver is submitted"
+ val masterUrl = startDummyServer(submitId = submittedDriverId, submitMessage = submitMessage)
+ val conf = new SparkConf(loadDefaults = false)
+ conf.set("spark.master", masterUrl)
+ conf.set("spark.app.name", "dreamer")
+ val appArgs = Array("one", "two", "six")
+ // main method calls this
+ val response = StandaloneRestClient.run("app-resource", "main-class", appArgs, conf)
+ val submitResponse = getSubmitResponse(response)
+ assert(submitResponse.action === Utils.getFormattedClassName(submitResponse))
+ assert(submitResponse.serverSparkVersion === SPARK_VERSION)
+ assert(submitResponse.message === submitMessage)
+ assert(submitResponse.submissionId === submittedDriverId)
+ assert(submitResponse.success)
+ }
+
+ test("kill submission") {
+ val submissionId = "my-lyft-driver"
+ val killMessage = "your driver is killed"
+ val masterUrl = startDummyServer(killMessage = killMessage)
+ val response = client.killSubmission(masterUrl, submissionId)
+ val killResponse = getKillResponse(response)
+ assert(killResponse.action === Utils.getFormattedClassName(killResponse))
+ assert(killResponse.serverSparkVersion === SPARK_VERSION)
+ assert(killResponse.message === killMessage)
+ assert(killResponse.submissionId === submissionId)
+ assert(killResponse.success)
+ }
+
+ test("request submission status") {
+ val submissionId = "my-uber-driver"
+ val submissionState = KILLED
+ val submissionException = new Exception("there was an irresponsible mix of alcohol and cars")
+ val masterUrl = startDummyServer(state = submissionState, exception = Some(submissionException))
+ val response = client.requestSubmissionStatus(masterUrl, submissionId)
+ val statusResponse = getStatusResponse(response)
+ assert(statusResponse.action === Utils.getFormattedClassName(statusResponse))
+ assert(statusResponse.serverSparkVersion === SPARK_VERSION)
+ assert(statusResponse.message.contains(submissionException.getMessage))
+ assert(statusResponse.submissionId === submissionId)
+ assert(statusResponse.driverState === submissionState.toString)
+ assert(statusResponse.success)
+ }
+
+ test("create then kill") {
+ val masterUrl = startSmartServer()
+ val request = constructSubmitRequest(masterUrl)
+ val response1 = client.createSubmission(masterUrl, request)
+ val submitResponse = getSubmitResponse(response1)
+ assert(submitResponse.success)
+ assert(submitResponse.submissionId != null)
+ // kill submission that was just created
+ val submissionId = submitResponse.submissionId
+ val response2 = client.killSubmission(masterUrl, submissionId)
+ val killResponse = getKillResponse(response2)
+ assert(killResponse.success)
+ assert(killResponse.submissionId === submissionId)
+ }
+
+ test("create then request status") {
+ val masterUrl = startSmartServer()
+ val request = constructSubmitRequest(masterUrl)
+ val response1 = client.createSubmission(masterUrl, request)
+ val submitResponse = getSubmitResponse(response1)
+ assert(submitResponse.success)
+ assert(submitResponse.submissionId != null)
+ // request status of submission that was just created
+ val submissionId = submitResponse.submissionId
+ val response2 = client.requestSubmissionStatus(masterUrl, submissionId)
+ val statusResponse = getStatusResponse(response2)
+ assert(statusResponse.success)
+ assert(statusResponse.submissionId === submissionId)
+ assert(statusResponse.driverState === RUNNING.toString)
+ }
+
+ test("create then kill then request status") {
+ val masterUrl = startSmartServer()
+ val request = constructSubmitRequest(masterUrl)
+ val response1 = client.createSubmission(masterUrl, request)
+ val response2 = client.createSubmission(masterUrl, request)
+ val submitResponse1 = getSubmitResponse(response1)
+ val submitResponse2 = getSubmitResponse(response2)
+ assert(submitResponse1.success)
+ assert(submitResponse2.success)
+ assert(submitResponse1.submissionId != null)
+ assert(submitResponse2.submissionId != null)
+ val submissionId1 = submitResponse1.submissionId
+ val submissionId2 = submitResponse2.submissionId
+ // kill only submission 1, but not submission 2
+ val response3 = client.killSubmission(masterUrl, submissionId1)
+ val killResponse = getKillResponse(response3)
+ assert(killResponse.success)
+ assert(killResponse.submissionId === submissionId1)
+ // request status for both submissions: 1 should be KILLED but 2 should be RUNNING still
+ val response4 = client.requestSubmissionStatus(masterUrl, submissionId1)
+ val response5 = client.requestSubmissionStatus(masterUrl, submissionId2)
+ val statusResponse1 = getStatusResponse(response4)
+ val statusResponse2 = getStatusResponse(response5)
+ assert(statusResponse1.submissionId === submissionId1)
+ assert(statusResponse2.submissionId === submissionId2)
+ assert(statusResponse1.driverState === KILLED.toString)
+ assert(statusResponse2.driverState === RUNNING.toString)
+ }
+
+ test("kill or request status before create") {
+ val masterUrl = startSmartServer()
+ val doesNotExist = "does-not-exist"
+ // kill a non-existent submission
+ val response1 = client.killSubmission(masterUrl, doesNotExist)
+ val killResponse = getKillResponse(response1)
+ assert(!killResponse.success)
+ assert(killResponse.submissionId === doesNotExist)
+ // request status for a non-existent submission
+ val response2 = client.requestSubmissionStatus(masterUrl, doesNotExist)
+ val statusResponse = getStatusResponse(response2)
+ assert(!statusResponse.success)
+ assert(statusResponse.submissionId === doesNotExist)
+ }
+
+ /* ---------------------------------------- *
+ | Aberrant client / server behavior |
+ * ---------------------------------------- */
+
+ test("good request paths") {
+ val masterUrl = startSmartServer()
+ val httpUrl = masterUrl.replace("spark://", "http://")
+ val v = StandaloneRestServer.PROTOCOL_VERSION
+ val json = constructSubmitRequest(masterUrl).toJson
+ val submitRequestPath = s"$httpUrl/$v/submissions/create"
+ val killRequestPath = s"$httpUrl/$v/submissions/kill"
+ val statusRequestPath = s"$httpUrl/$v/submissions/status"
+ val (response1, code1) = sendHttpRequestWithResponse(submitRequestPath, "POST", json)
+ val (response2, code2) = sendHttpRequestWithResponse(s"$killRequestPath/anything", "POST")
+ val (response3, code3) = sendHttpRequestWithResponse(s"$killRequestPath/any/thing", "POST")
+ val (response4, code4) = sendHttpRequestWithResponse(s"$statusRequestPath/anything", "GET")
+ val (response5, code5) = sendHttpRequestWithResponse(s"$statusRequestPath/any/thing", "GET")
+ // these should all succeed and the responses should be of the correct types
+ getSubmitResponse(response1)
+ val killResponse1 = getKillResponse(response2)
+ val killResponse2 = getKillResponse(response3)
+ val statusResponse1 = getStatusResponse(response4)
+ val statusResponse2 = getStatusResponse(response5)
+ assert(killResponse1.submissionId === "anything")
+ assert(killResponse2.submissionId === "any")
+ assert(statusResponse1.submissionId === "anything")
+ assert(statusResponse2.submissionId === "any")
+ assert(code1 === HttpServletResponse.SC_OK)
+ assert(code2 === HttpServletResponse.SC_OK)
+ assert(code3 === HttpServletResponse.SC_OK)
+ assert(code4 === HttpServletResponse.SC_OK)
+ assert(code5 === HttpServletResponse.SC_OK)
+ }
+
+ test("good request paths, bad requests") {
+ val masterUrl = startSmartServer()
+ val httpUrl = masterUrl.replace("spark://", "http://")
+ val v = StandaloneRestServer.PROTOCOL_VERSION
+ val submitRequestPath = s"$httpUrl/$v/submissions/create"
+ val killRequestPath = s"$httpUrl/$v/submissions/kill"
+ val statusRequestPath = s"$httpUrl/$v/submissions/status"
+ val goodJson = constructSubmitRequest(masterUrl).toJson
+ val badJson1 = goodJson.replaceAll("action", "fraction") // invalid JSON
+ val badJson2 = goodJson.substring(goodJson.size / 2) // malformed JSON
+ val notJson = "\"hello, world\""
+ val (response1, code1) = sendHttpRequestWithResponse(submitRequestPath, "POST") // missing JSON
+ val (response2, code2) = sendHttpRequestWithResponse(submitRequestPath, "POST", badJson1)
+ val (response3, code3) = sendHttpRequestWithResponse(submitRequestPath, "POST", badJson2)
+ val (response4, code4) = sendHttpRequestWithResponse(killRequestPath, "POST") // missing ID
+ val (response5, code5) = sendHttpRequestWithResponse(s"$killRequestPath/", "POST")
+ val (response6, code6) = sendHttpRequestWithResponse(statusRequestPath, "GET") // missing ID
+ val (response7, code7) = sendHttpRequestWithResponse(s"$statusRequestPath/", "GET")
+ val (response8, code8) = sendHttpRequestWithResponse(submitRequestPath, "POST", notJson)
+ // these should all fail as error responses
+ getErrorResponse(response1)
+ getErrorResponse(response2)
+ getErrorResponse(response3)
+ getErrorResponse(response4)
+ getErrorResponse(response5)
+ getErrorResponse(response6)
+ getErrorResponse(response7)
+ getErrorResponse(response8)
+ assert(code1 === HttpServletResponse.SC_BAD_REQUEST)
+ assert(code2 === HttpServletResponse.SC_BAD_REQUEST)
+ assert(code3 === HttpServletResponse.SC_BAD_REQUEST)
+ assert(code4 === HttpServletResponse.SC_BAD_REQUEST)
+ assert(code5 === HttpServletResponse.SC_BAD_REQUEST)
+ assert(code6 === HttpServletResponse.SC_BAD_REQUEST)
+ assert(code7 === HttpServletResponse.SC_BAD_REQUEST)
+ assert(code8 === HttpServletResponse.SC_BAD_REQUEST)
+ }
+
+ test("bad request paths") {
+ val masterUrl = startSmartServer()
+ val httpUrl = masterUrl.replace("spark://", "http://")
+ val v = StandaloneRestServer.PROTOCOL_VERSION
+ val (response1, code1) = sendHttpRequestWithResponse(httpUrl, "GET")
+ val (response2, code2) = sendHttpRequestWithResponse(s"$httpUrl/", "GET")
+ val (response3, code3) = sendHttpRequestWithResponse(s"$httpUrl/$v", "GET")
+ val (response4, code4) = sendHttpRequestWithResponse(s"$httpUrl/$v/", "GET")
+ val (response5, code5) = sendHttpRequestWithResponse(s"$httpUrl/$v/submissions", "GET")
+ val (response6, code6) = sendHttpRequestWithResponse(s"$httpUrl/$v/submissions/", "GET")
+ val (response7, code7) = sendHttpRequestWithResponse(s"$httpUrl/$v/submissions/bad", "GET")
+ val (response8, code8) = sendHttpRequestWithResponse(s"$httpUrl/bad-version", "GET")
+ assert(code1 === HttpServletResponse.SC_BAD_REQUEST)
+ assert(code2 === HttpServletResponse.SC_BAD_REQUEST)
+ assert(code3 === HttpServletResponse.SC_BAD_REQUEST)
+ assert(code4 === HttpServletResponse.SC_BAD_REQUEST)
+ assert(code5 === HttpServletResponse.SC_BAD_REQUEST)
+ assert(code6 === HttpServletResponse.SC_BAD_REQUEST)
+ assert(code7 === HttpServletResponse.SC_BAD_REQUEST)
+ assert(code8 === StandaloneRestServer.SC_UNKNOWN_PROTOCOL_VERSION)
+ // all responses should be error responses
+ val errorResponse1 = getErrorResponse(response1)
+ val errorResponse2 = getErrorResponse(response2)
+ val errorResponse3 = getErrorResponse(response3)
+ val errorResponse4 = getErrorResponse(response4)
+ val errorResponse5 = getErrorResponse(response5)
+ val errorResponse6 = getErrorResponse(response6)
+ val errorResponse7 = getErrorResponse(response7)
+ val errorResponse8 = getErrorResponse(response8)
+ // only the incompatible version response should have server protocol version set
+ assert(errorResponse1.highestProtocolVersion === null)
+ assert(errorResponse2.highestProtocolVersion === null)
+ assert(errorResponse3.highestProtocolVersion === null)
+ assert(errorResponse4.highestProtocolVersion === null)
+ assert(errorResponse5.highestProtocolVersion === null)
+ assert(errorResponse6.highestProtocolVersion === null)
+ assert(errorResponse7.highestProtocolVersion === null)
+ assert(errorResponse8.highestProtocolVersion === StandaloneRestServer.PROTOCOL_VERSION)
+ }
+
+ test("server returns unknown fields") {
+ val masterUrl = startSmartServer()
+ val httpUrl = masterUrl.replace("spark://", "http://")
+ val v = StandaloneRestServer.PROTOCOL_VERSION
+ val submitRequestPath = s"$httpUrl/$v/submissions/create"
+ val oldJson = constructSubmitRequest(masterUrl).toJson
+ val oldFields = parse(oldJson).asInstanceOf[JObject].obj
+ val newFields = oldFields ++ Seq(
+ JField("tomato", JString("not-a-fruit")),
+ JField("potato", JString("not-po-tah-to"))
+ )
+ val newJson = pretty(render(JObject(newFields)))
+ // send two requests, one with the unknown fields and the other without
+ val (response1, code1) = sendHttpRequestWithResponse(submitRequestPath, "POST", oldJson)
+ val (response2, code2) = sendHttpRequestWithResponse(submitRequestPath, "POST", newJson)
+ val submitResponse1 = getSubmitResponse(response1)
+ val submitResponse2 = getSubmitResponse(response2)
+ assert(code1 === HttpServletResponse.SC_OK)
+ assert(code2 === HttpServletResponse.SC_OK)
+ // only the response to the modified request should have unknown fields set
+ assert(submitResponse1.unknownFields === null)
+ assert(submitResponse2.unknownFields === Array("tomato", "potato"))
+ }
+
+ test("client handles faulty server") {
+ val masterUrl = startFaultyServer()
+ val httpUrl = masterUrl.replace("spark://", "http://")
+ val v = StandaloneRestServer.PROTOCOL_VERSION
+ val submitRequestPath = s"$httpUrl/$v/submissions/create"
+ val killRequestPath = s"$httpUrl/$v/submissions/kill/anything"
+ val statusRequestPath = s"$httpUrl/$v/submissions/status/anything"
+ val json = constructSubmitRequest(masterUrl).toJson
+ // server returns malformed response unwittingly
+ // client should throw an appropriate exception to indicate server failure
+ val conn1 = sendHttpRequest(submitRequestPath, "POST", json)
+ intercept[SubmitRestProtocolException] { client.readResponse(conn1) }
+ // server attempts to send invalid response, but fails internally on validation
+ // client should receive an error response as server is able to recover
+ val conn2 = sendHttpRequest(killRequestPath, "POST")
+ val response2 = client.readResponse(conn2)
+ getErrorResponse(response2)
+ assert(conn2.getResponseCode === HttpServletResponse.SC_INTERNAL_SERVER_ERROR)
+ // server explodes internally beyond recovery
+ // client should throw an appropriate exception to indicate server failure
+ val conn3 = sendHttpRequest(statusRequestPath, "GET")
+ intercept[SubmitRestProtocolException] { client.readResponse(conn3) } // empty response
+ assert(conn3.getResponseCode === HttpServletResponse.SC_INTERNAL_SERVER_ERROR)
+ }
+
+ /* --------------------- *
+ | Helper methods |
+ * --------------------- */
+
+ /** Start a dummy server that responds to requests using the specified parameters. */
+ private def startDummyServer(
+ submitId: String = "fake-driver-id",
+ submitMessage: String = "driver is submitted",
+ killMessage: String = "driver is killed",
+ state: DriverState = FINISHED,
+ exception: Option[Exception] = None): String = {
+ startServer(new DummyMaster(submitId, submitMessage, killMessage, state, exception))
+ }
+
+ /** Start a smarter dummy server that keeps track of submitted driver states. */
+ private def startSmartServer(): String = {
+ startServer(new SmarterMaster)
+ }
+
+ /** Start a dummy server that is faulty in many ways... */
+ private def startFaultyServer(): String = {
+ startServer(new DummyMaster, faulty = true)
+ }
+
+ /**
+ * Start a [[StandaloneRestServer]] that communicates with the given actor.
+ * If `faulty` is true, start an [[FaultyStandaloneRestServer]] instead.
+ * Return the master URL that corresponds to the address of this server.
+ */
+ private def startServer(makeFakeMaster: => Actor, faulty: Boolean = false): String = {
+ val name = "test-standalone-rest-protocol"
+ val conf = new SparkConf
+ val localhost = Utils.localHostName()
+ val securityManager = new SecurityManager(conf)
+ val (_actorSystem, _) = AkkaUtils.createActorSystem(name, localhost, 0, conf, securityManager)
+ val fakeMasterRef = _actorSystem.actorOf(Props(makeFakeMaster))
+ val _server =
+ if (faulty) {
+ new FaultyStandaloneRestServer(localhost, 0, fakeMasterRef, "spark://fake:7077", conf)
+ } else {
+ new StandaloneRestServer(localhost, 0, fakeMasterRef, "spark://fake:7077", conf)
+ }
+ val port = _server.start()
+ // set these to clean them up after every test
+ actorSystem = Some(_actorSystem)
+ server = Some(_server)
+ s"spark://$localhost:$port"
+ }
+
+ /** Create a submit request with real parameters using Spark submit. */
+ private def constructSubmitRequest(
+ masterUrl: String,
+ appArgs: Array[String] = Array.empty): CreateSubmissionRequest = {
+ val mainClass = "main-class-not-used"
+ val mainJar = "dummy-jar-not-used.jar"
+ val commandLineArgs = Array(
+ "--deploy-mode", "cluster",
+ "--master", masterUrl,
+ "--name", mainClass,
+ "--class", mainClass,
+ mainJar) ++ appArgs
+ val args = new SparkSubmitArguments(commandLineArgs)
+ val (_, _, sparkProperties, _) = SparkSubmit.prepareSubmitEnvironment(args)
+ client.constructSubmitRequest(
+ mainJar, mainClass, appArgs, sparkProperties.toMap, Map.empty)
+ }
+
+ /** Return the response as a submit response, or fail with error otherwise. */
+ private def getSubmitResponse(response: SubmitRestProtocolResponse): CreateSubmissionResponse = {
+ response match {
+ case s: CreateSubmissionResponse => s
+ case e: ErrorResponse => fail(s"Server returned error: ${e.message}")
+ case r => fail(s"Expected submit response. Actual: ${r.toJson}")
+ }
+ }
+
+ /** Return the response as a kill response, or fail with error otherwise. */
+ private def getKillResponse(response: SubmitRestProtocolResponse): KillSubmissionResponse = {
+ response match {
+ case k: KillSubmissionResponse => k
+ case e: ErrorResponse => fail(s"Server returned error: ${e.message}")
+ case r => fail(s"Expected kill response. Actual: ${r.toJson}")
+ }
+ }
+
+ /** Return the response as a status response, or fail with error otherwise. */
+ private def getStatusResponse(response: SubmitRestProtocolResponse): SubmissionStatusResponse = {
+ response match {
+ case s: SubmissionStatusResponse => s
+ case e: ErrorResponse => fail(s"Server returned error: ${e.message}")
+ case r => fail(s"Expected status response. Actual: ${r.toJson}")
+ }
+ }
+
+ /** Return the response as an error response, or fail if the response was not an error. */
+ private def getErrorResponse(response: SubmitRestProtocolResponse): ErrorResponse = {
+ response match {
+ case e: ErrorResponse => e
+ case r => fail(s"Expected error response. Actual: ${r.toJson}")
+ }
+ }
+
+ /**
+ * Send an HTTP request to the given URL using the method and the body specified.
+ * Return the connection object.
+ */
+ private def sendHttpRequest(
+ url: String,
+ method: String,
+ body: String = ""): HttpURLConnection = {
+ val conn = new URL(url).openConnection().asInstanceOf[HttpURLConnection]
+ conn.setRequestMethod(method)
+ if (body.nonEmpty) {
+ conn.setDoOutput(true)
+ val out = new DataOutputStream(conn.getOutputStream)
+ out.write(body.getBytes(Charsets.UTF_8))
+ out.close()
+ }
+ conn
+ }
+
+ /**
+ * Send an HTTP request to the given URL using the method and the body specified.
+ * Return a 2-tuple of the response message from the server and the response code.
+ */
+ private def sendHttpRequestWithResponse(
+ url: String,
+ method: String,
+ body: String = ""): (SubmitRestProtocolResponse, Int) = {
+ val conn = sendHttpRequest(url, method, body)
+ (client.readResponse(conn), conn.getResponseCode)
+ }
+}
+
+/**
+ * A mock standalone Master that responds with dummy messages.
+ * In all responses, the success parameter is always true.
+ */
+private class DummyMaster(
+ submitId: String = "fake-driver-id",
+ submitMessage: String = "submitted",
+ killMessage: String = "killed",
+ state: DriverState = FINISHED,
+ exception: Option[Exception] = None)
+ extends Actor {
+
+ override def receive = {
+ case RequestSubmitDriver(driverDesc) =>
+ sender ! SubmitDriverResponse(success = true, Some(submitId), submitMessage)
+ case RequestKillDriver(driverId) =>
+ sender ! KillDriverResponse(driverId, success = true, killMessage)
+ case RequestDriverStatus(driverId) =>
+ sender ! DriverStatusResponse(found = true, Some(state), None, None, exception)
+ }
+}
+
+/**
+ * A mock standalone Master that keeps track of drivers that have been submitted.
+ *
+ * If a driver is submitted, its state is immediately set to RUNNING.
+ * If an existing driver is killed, its state is immediately set to KILLED.
+ * If an existing driver's status is requested, its state is returned in the response.
+ * Submits are always successful while kills and status requests are successful only
+ * if the driver was submitted in the past.
+ */
+private class SmarterMaster extends Actor {
+ private var counter: Int = 0
+ private val submittedDrivers = new mutable.HashMap[String, DriverState]
+
+ override def receive = {
+ case RequestSubmitDriver(driverDesc) =>
+ val driverId = s"driver-$counter"
+ submittedDrivers(driverId) = RUNNING
+ counter += 1
+ sender ! SubmitDriverResponse(success = true, Some(driverId), "submitted")
+
+ case RequestKillDriver(driverId) =>
+ val success = submittedDrivers.contains(driverId)
+ if (success) {
+ submittedDrivers(driverId) = KILLED
+ }
+ sender ! KillDriverResponse(driverId, success, "killed")
+
+ case RequestDriverStatus(driverId) =>
+ val found = submittedDrivers.contains(driverId)
+ val state = submittedDrivers.get(driverId)
+ sender ! DriverStatusResponse(found, state, None, None, None)
+ }
+}
+
+/**
+ * A [[StandaloneRestServer]] that is faulty in many ways.
+ *
+ * When handling a submit request, the server returns a malformed JSON.
+ * When handling a kill request, the server returns an invalid JSON.
+ * When handling a status request, the server throws an internal exception.
+ * The purpose of this class is to test that client handles these cases gracefully.
+ */
+private class FaultyStandaloneRestServer(
+ host: String,
+ requestedPort: Int,
+ masterActor: ActorRef,
+ masterUrl: String,
+ masterConf: SparkConf)
+ extends StandaloneRestServer(host, requestedPort, masterActor, masterUrl, masterConf) {
+
+ protected override val contextToServlet = Map[String, StandaloneRestServlet](
+ s"$baseContext/create/*" -> new MalformedSubmitServlet,
+ s"$baseContext/kill/*" -> new InvalidKillServlet,
+ s"$baseContext/status/*" -> new ExplodingStatusServlet,
+ "/*" -> new ErrorServlet
+ )
+
+ /** A faulty servlet that produces malformed responses. */
+ class MalformedSubmitServlet extends SubmitRequestServlet(masterActor, masterUrl, masterConf) {
+ protected override def sendResponse(
+ responseMessage: SubmitRestProtocolResponse,
+ responseServlet: HttpServletResponse): Unit = {
+ val badJson = responseMessage.toJson.drop(10).dropRight(20)
+ responseServlet.getWriter.write(badJson)
+ }
+ }
+
+ /** A faulty servlet that produces invalid responses. */
+ class InvalidKillServlet extends KillRequestServlet(masterActor, masterConf) {
+ protected override def handleKill(submissionId: String): KillSubmissionResponse = {
+ val k = super.handleKill(submissionId)
+ k.submissionId = null
+ k
+ }
+ }
+
+ /** A faulty status servlet that explodes. */
+ class ExplodingStatusServlet extends StatusRequestServlet(masterActor, masterConf) {
+ private def explode: Int = 1 / 0
+ protected override def handleStatus(submissionId: String): SubmissionStatusResponse = {
+ val s = super.handleStatus(submissionId)
+ s.workerId = explode.toString
+ s
+ }
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala
new file mode 100644
index 0000000000000..1d64ec201e647
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala
@@ -0,0 +1,324 @@
+/*
+ * 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.rest
+
+import java.lang.Boolean
+import java.lang.Integer
+
+import org.json4s.jackson.JsonMethods._
+import org.scalatest.FunSuite
+
+import org.apache.spark.SparkConf
+
+/**
+ * Tests for the REST application submission protocol.
+ */
+class SubmitRestProtocolSuite extends FunSuite {
+
+ test("validate") {
+ val request = new DummyRequest
+ intercept[SubmitRestProtocolException] { request.validate() } // missing everything
+ request.clientSparkVersion = "1.2.3"
+ intercept[SubmitRestProtocolException] { request.validate() } // missing name and age
+ request.name = "something"
+ intercept[SubmitRestProtocolException] { request.validate() } // missing only age
+ request.age = 2
+ intercept[SubmitRestProtocolException] { request.validate() } // age too low
+ request.age = 10
+ request.validate() // everything is set properly
+ request.clientSparkVersion = null
+ intercept[SubmitRestProtocolException] { request.validate() } // missing only Spark version
+ request.clientSparkVersion = "1.2.3"
+ request.name = null
+ intercept[SubmitRestProtocolException] { request.validate() } // missing only name
+ request.message = "not-setting-name"
+ intercept[SubmitRestProtocolException] { request.validate() } // still missing name
+ }
+
+ test("request to and from JSON") {
+ val request = new DummyRequest
+ intercept[SubmitRestProtocolException] { request.toJson } // implicit validation
+ request.clientSparkVersion = "1.2.3"
+ request.active = true
+ request.age = 25
+ request.name = "jung"
+ val json = request.toJson
+ assertJsonEquals(json, dummyRequestJson)
+ val newRequest = SubmitRestProtocolMessage.fromJson(json, classOf[DummyRequest])
+ assert(newRequest.clientSparkVersion === "1.2.3")
+ assert(newRequest.clientSparkVersion === "1.2.3")
+ assert(newRequest.active)
+ assert(newRequest.age === 25)
+ assert(newRequest.name === "jung")
+ assert(newRequest.message === null)
+ }
+
+ test("response to and from JSON") {
+ val response = new DummyResponse
+ response.serverSparkVersion = "3.3.4"
+ response.success = true
+ val json = response.toJson
+ assertJsonEquals(json, dummyResponseJson)
+ val newResponse = SubmitRestProtocolMessage.fromJson(json, classOf[DummyResponse])
+ assert(newResponse.serverSparkVersion === "3.3.4")
+ assert(newResponse.serverSparkVersion === "3.3.4")
+ assert(newResponse.success)
+ assert(newResponse.message === null)
+ }
+
+ test("CreateSubmissionRequest") {
+ val message = new CreateSubmissionRequest
+ intercept[SubmitRestProtocolException] { message.validate() }
+ message.clientSparkVersion = "1.2.3"
+ message.appResource = "honey-walnut-cherry.jar"
+ message.mainClass = "org.apache.spark.examples.SparkPie"
+ val conf = new SparkConf(false)
+ conf.set("spark.app.name", "SparkPie")
+ message.sparkProperties = conf.getAll.toMap
+ message.validate()
+ // optional fields
+ conf.set("spark.jars", "mayonnaise.jar,ketchup.jar")
+ conf.set("spark.files", "fireball.png")
+ conf.set("spark.driver.memory", "512m")
+ conf.set("spark.driver.cores", "180")
+ conf.set("spark.driver.extraJavaOptions", " -Dslices=5 -Dcolor=mostly_red")
+ conf.set("spark.driver.extraClassPath", "food-coloring.jar")
+ conf.set("spark.driver.extraLibraryPath", "pickle.jar")
+ conf.set("spark.driver.supervise", "false")
+ conf.set("spark.executor.memory", "256m")
+ conf.set("spark.cores.max", "10000")
+ message.sparkProperties = conf.getAll.toMap
+ message.appArgs = Array("two slices", "a hint of cinnamon")
+ message.environmentVariables = Map("PATH" -> "/dev/null")
+ message.validate()
+ // bad fields
+ var badConf = conf.clone().set("spark.driver.cores", "one hundred feet")
+ message.sparkProperties = badConf.getAll.toMap
+ intercept[SubmitRestProtocolException] { message.validate() }
+ badConf = conf.clone().set("spark.driver.supervise", "nope, never")
+ message.sparkProperties = badConf.getAll.toMap
+ intercept[SubmitRestProtocolException] { message.validate() }
+ badConf = conf.clone().set("spark.cores.max", "two men")
+ message.sparkProperties = badConf.getAll.toMap
+ intercept[SubmitRestProtocolException] { message.validate() }
+ message.sparkProperties = conf.getAll.toMap
+ // test JSON
+ val json = message.toJson
+ assertJsonEquals(json, submitDriverRequestJson)
+ val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[CreateSubmissionRequest])
+ assert(newMessage.clientSparkVersion === "1.2.3")
+ assert(newMessage.appResource === "honey-walnut-cherry.jar")
+ assert(newMessage.mainClass === "org.apache.spark.examples.SparkPie")
+ assert(newMessage.sparkProperties("spark.app.name") === "SparkPie")
+ assert(newMessage.sparkProperties("spark.jars") === "mayonnaise.jar,ketchup.jar")
+ assert(newMessage.sparkProperties("spark.files") === "fireball.png")
+ assert(newMessage.sparkProperties("spark.driver.memory") === "512m")
+ assert(newMessage.sparkProperties("spark.driver.cores") === "180")
+ assert(newMessage.sparkProperties("spark.driver.extraJavaOptions") === " -Dslices=5 -Dcolor=mostly_red")
+ assert(newMessage.sparkProperties("spark.driver.extraClassPath") === "food-coloring.jar")
+ assert(newMessage.sparkProperties("spark.driver.extraLibraryPath") === "pickle.jar")
+ assert(newMessage.sparkProperties("spark.driver.supervise") === "false")
+ assert(newMessage.sparkProperties("spark.executor.memory") === "256m")
+ assert(newMessage.sparkProperties("spark.cores.max") === "10000")
+ assert(newMessage.appArgs === message.appArgs)
+ assert(newMessage.sparkProperties === message.sparkProperties)
+ assert(newMessage.environmentVariables === message.environmentVariables)
+ }
+
+ test("CreateSubmissionResponse") {
+ val message = new CreateSubmissionResponse
+ intercept[SubmitRestProtocolException] { message.validate() }
+ message.serverSparkVersion = "1.2.3"
+ message.submissionId = "driver_123"
+ message.success = true
+ message.validate()
+ // test JSON
+ val json = message.toJson
+ assertJsonEquals(json, submitDriverResponseJson)
+ val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[CreateSubmissionResponse])
+ assert(newMessage.serverSparkVersion === "1.2.3")
+ assert(newMessage.submissionId === "driver_123")
+ assert(newMessage.success)
+ }
+
+ test("KillSubmissionResponse") {
+ val message = new KillSubmissionResponse
+ intercept[SubmitRestProtocolException] { message.validate() }
+ message.serverSparkVersion = "1.2.3"
+ message.submissionId = "driver_123"
+ message.success = true
+ message.validate()
+ // test JSON
+ val json = message.toJson
+ assertJsonEquals(json, killDriverResponseJson)
+ val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[KillSubmissionResponse])
+ assert(newMessage.serverSparkVersion === "1.2.3")
+ assert(newMessage.submissionId === "driver_123")
+ assert(newMessage.success)
+ }
+
+ test("SubmissionStatusResponse") {
+ val message = new SubmissionStatusResponse
+ intercept[SubmitRestProtocolException] { message.validate() }
+ message.serverSparkVersion = "1.2.3"
+ message.submissionId = "driver_123"
+ message.success = true
+ message.validate()
+ // optional fields
+ message.driverState = "RUNNING"
+ message.workerId = "worker_123"
+ message.workerHostPort = "1.2.3.4:7780"
+ // test JSON
+ val json = message.toJson
+ assertJsonEquals(json, driverStatusResponseJson)
+ val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[SubmissionStatusResponse])
+ assert(newMessage.serverSparkVersion === "1.2.3")
+ assert(newMessage.submissionId === "driver_123")
+ assert(newMessage.driverState === "RUNNING")
+ assert(newMessage.success)
+ assert(newMessage.workerId === "worker_123")
+ assert(newMessage.workerHostPort === "1.2.3.4:7780")
+ }
+
+ test("ErrorResponse") {
+ val message = new ErrorResponse
+ intercept[SubmitRestProtocolException] { message.validate() }
+ message.serverSparkVersion = "1.2.3"
+ message.message = "Field not found in submit request: X"
+ message.validate()
+ // test JSON
+ val json = message.toJson
+ assertJsonEquals(json, errorJson)
+ val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[ErrorResponse])
+ assert(newMessage.serverSparkVersion === "1.2.3")
+ assert(newMessage.message === "Field not found in submit request: X")
+ }
+
+ private val dummyRequestJson =
+ """
+ |{
+ | "action" : "DummyRequest",
+ | "active" : true,
+ | "age" : 25,
+ | "clientSparkVersion" : "1.2.3",
+ | "name" : "jung"
+ |}
+ """.stripMargin
+
+ private val dummyResponseJson =
+ """
+ |{
+ | "action" : "DummyResponse",
+ | "serverSparkVersion" : "3.3.4",
+ | "success": true
+ |}
+ """.stripMargin
+
+ private val submitDriverRequestJson =
+ """
+ |{
+ | "action" : "CreateSubmissionRequest",
+ | "appArgs" : [ "two slices", "a hint of cinnamon" ],
+ | "appResource" : "honey-walnut-cherry.jar",
+ | "clientSparkVersion" : "1.2.3",
+ | "environmentVariables" : {
+ | "PATH" : "/dev/null"
+ | },
+ | "mainClass" : "org.apache.spark.examples.SparkPie",
+ | "sparkProperties" : {
+ | "spark.driver.extraLibraryPath" : "pickle.jar",
+ | "spark.jars" : "mayonnaise.jar,ketchup.jar",
+ | "spark.driver.supervise" : "false",
+ | "spark.app.name" : "SparkPie",
+ | "spark.cores.max" : "10000",
+ | "spark.driver.memory" : "512m",
+ | "spark.files" : "fireball.png",
+ | "spark.driver.cores" : "180",
+ | "spark.driver.extraJavaOptions" : " -Dslices=5 -Dcolor=mostly_red",
+ | "spark.executor.memory" : "256m",
+ | "spark.driver.extraClassPath" : "food-coloring.jar"
+ | }
+ |}
+ """.stripMargin
+
+ private val submitDriverResponseJson =
+ """
+ |{
+ | "action" : "CreateSubmissionResponse",
+ | "serverSparkVersion" : "1.2.3",
+ | "submissionId" : "driver_123",
+ | "success" : true
+ |}
+ """.stripMargin
+
+ private val killDriverResponseJson =
+ """
+ |{
+ | "action" : "KillSubmissionResponse",
+ | "serverSparkVersion" : "1.2.3",
+ | "submissionId" : "driver_123",
+ | "success" : true
+ |}
+ """.stripMargin
+
+ private val driverStatusResponseJson =
+ """
+ |{
+ | "action" : "SubmissionStatusResponse",
+ | "driverState" : "RUNNING",
+ | "serverSparkVersion" : "1.2.3",
+ | "submissionId" : "driver_123",
+ | "success" : true,
+ | "workerHostPort" : "1.2.3.4:7780",
+ | "workerId" : "worker_123"
+ |}
+ """.stripMargin
+
+ private val errorJson =
+ """
+ |{
+ | "action" : "ErrorResponse",
+ | "message" : "Field not found in submit request: X",
+ | "serverSparkVersion" : "1.2.3"
+ |}
+ """.stripMargin
+
+ /** Assert that the contents in the two JSON strings are equal after ignoring whitespace. */
+ private def assertJsonEquals(jsonString1: String, jsonString2: String): Unit = {
+ val trimmedJson1 = jsonString1.trim
+ val trimmedJson2 = jsonString2.trim
+ val json1 = compact(render(parse(trimmedJson1)))
+ val json2 = compact(render(parse(trimmedJson2)))
+ // Put this on a separate line to avoid printing comparison twice when test fails
+ val equals = json1 == json2
+ assert(equals, "\"[%s]\" did not equal \"[%s]\"".format(trimmedJson1, trimmedJson2))
+ }
+}
+
+private class DummyResponse extends SubmitRestProtocolResponse
+private class DummyRequest extends SubmitRestProtocolRequest {
+ var active: Boolean = null
+ var age: Integer = null
+ var name: String = null
+ protected override def doValidate(): Unit = {
+ super.doValidate()
+ assertFieldIsSet(name, "name")
+ assertFieldIsSet(age, "age")
+ assert(age > 5, "Not old enough!")
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala
similarity index 94%
rename from core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala
rename to core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala
index 7915ee75d8778..1c27d83cf876c 100644
--- a/core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala
@@ -15,11 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark.deploy
+package org.apache.spark.deploy.worker
-import org.apache.spark.deploy.worker.CommandUtils
+import org.apache.spark.deploy.Command
import org.apache.spark.util.Utils
-
import org.scalatest.{FunSuite, Matchers}
class CommandUtilsSuite extends FunSuite with Matchers {
diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala
index b6f4411e0587a..aa6e4874cecde 100644
--- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala
@@ -27,6 +27,7 @@ import org.scalatest.FunSuite
import org.apache.spark.SparkConf
import org.apache.spark.deploy.{Command, DriverDescription}
+import org.apache.spark.util.Clock
class DriverRunnerTest extends FunSuite {
private def createDriverRunner() = {
@@ -129,7 +130,7 @@ class DriverRunnerTest extends FunSuite {
.thenReturn(-1) // fail 3
.thenReturn(-1) // fail 4
.thenReturn(0) // success
- when(clock.currentTimeMillis())
+ when(clock.getTimeMillis())
.thenReturn(0).thenReturn(1000) // fail 1 (short)
.thenReturn(1000).thenReturn(2000) // fail 2 (short)
.thenReturn(2000).thenReturn(10000) // fail 3 (long)
diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
index 6f233d7cf97aa..6fca6321e5a1b 100644
--- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
@@ -32,8 +32,8 @@ class ExecutorRunnerTest extends FunSuite {
val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
val appDesc = new ApplicationDescription("app name", Some(8), 500,
Command("foo", Seq(appId), Map(), Seq(), Seq(), Seq()), "appUiUrl")
- val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321",
- new File(sparkHome), new File("ooga"), "blah", new SparkConf, Seq("localDir"),
+ val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", 123,
+ "publicAddr", new File(sparkHome), new File("ooga"), "blah", new SparkConf, Seq("localDir"),
ExecutorState.RUNNING)
val builder = CommandUtils.buildProcessBuilder(appDesc.command, 512, sparkHome, er.substituteVariables)
assert(builder.command().last === appId)
diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala
new file mode 100644
index 0000000000000..84e2fd7ad936d
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala
@@ -0,0 +1,57 @@
+/*
+ * 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.worker
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.Command
+
+import org.scalatest.{Matchers, FunSuite}
+
+class WorkerSuite extends FunSuite with Matchers {
+
+ def cmd(javaOpts: String*) = Command("", Seq.empty, Map.empty, Seq.empty, Seq.empty, Seq(javaOpts:_*))
+ def conf(opts: (String, String)*) = new SparkConf(loadDefaults = false).setAll(opts)
+
+ test("test isUseLocalNodeSSLConfig") {
+ Worker.isUseLocalNodeSSLConfig(cmd("-Dasdf=dfgh")) shouldBe false
+ Worker.isUseLocalNodeSSLConfig(cmd("-Dspark.ssl.useNodeLocalConf=true")) shouldBe true
+ Worker.isUseLocalNodeSSLConfig(cmd("-Dspark.ssl.useNodeLocalConf=false")) shouldBe false
+ Worker.isUseLocalNodeSSLConfig(cmd("-Dspark.ssl.useNodeLocalConf=")) shouldBe false
+ }
+
+ test("test maybeUpdateSSLSettings") {
+ Worker.maybeUpdateSSLSettings(
+ cmd("-Dasdf=dfgh", "-Dspark.ssl.opt1=x"),
+ conf("spark.ssl.opt1" -> "y", "spark.ssl.opt2" -> "z"))
+ .javaOpts should contain theSameElementsInOrderAs Seq(
+ "-Dasdf=dfgh", "-Dspark.ssl.opt1=x")
+
+ Worker.maybeUpdateSSLSettings(
+ cmd("-Dspark.ssl.useNodeLocalConf=false", "-Dspark.ssl.opt1=x"),
+ conf("spark.ssl.opt1" -> "y", "spark.ssl.opt2" -> "z"))
+ .javaOpts should contain theSameElementsInOrderAs Seq(
+ "-Dspark.ssl.useNodeLocalConf=false", "-Dspark.ssl.opt1=x")
+
+ Worker.maybeUpdateSSLSettings(
+ cmd("-Dspark.ssl.useNodeLocalConf=true", "-Dspark.ssl.opt1=x"),
+ conf("spark.ssl.opt1" -> "y", "spark.ssl.opt2" -> "z"))
+ .javaOpts should contain theSameElementsAs Seq(
+ "-Dspark.ssl.useNodeLocalConf=true", "-Dspark.ssl.opt1=y", "-Dspark.ssl.opt2=z")
+
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala
new file mode 100644
index 0000000000000..326e203afe136
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.executor
+
+import org.scalatest.FunSuite
+
+class TaskMetricsSuite extends FunSuite {
+ test("[SPARK-5701] updateShuffleReadMetrics: ShuffleReadMetrics not added when no shuffle deps") {
+ val taskMetrics = new TaskMetrics()
+ taskMetrics.updateShuffleReadMetrics()
+ assert(taskMetrics.shuffleReadMetrics.isEmpty)
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala
index 98b0a16ce88ba..2e58c159a2ed8 100644
--- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala
+++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala
@@ -28,7 +28,7 @@ import org.scalatest.FunSuite
import org.apache.hadoop.io.Text
-import org.apache.spark.SparkContext
+import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.util.Utils
import org.apache.hadoop.io.compress.{DefaultCodec, CompressionCodecFactory, GzipCodec}
@@ -42,7 +42,15 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll {
private var factory: CompressionCodecFactory = _
override def beforeAll() {
- sc = new SparkContext("local", "test")
+ // Hadoop's FileSystem caching does not use the Configuration as part of its cache key, which
+ // can cause Filesystem.get(Configuration) to return a cached instance created with a different
+ // configuration than the one passed to get() (see HADOOP-8490 for more details). This caused
+ // hard-to-reproduce test failures, since any suites that were run after this one would inherit
+ // the new value of "fs.local.block.size" (see SPARK-5227 and SPARK-5679). To work around this,
+ // we disable FileSystem caching in this suite.
+ val conf = new SparkConf().set("spark.hadoop.fs.file.impl.disable.cache", "true")
+
+ sc = new SparkContext("local", "test", conf)
// Set the block size of local file system to test whether files are split right or not.
sc.hadoopConfiguration.setLong("fs.local.block.size", 32)
diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala
index 81db66ae17464..78fa98a3b9065 100644
--- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala
@@ -21,44 +21,46 @@ import java.io.{File, FileWriter, PrintWriter}
import scala.collection.mutable.ArrayBuffer
-import org.scalatest.FunSuite
-
+import org.apache.commons.lang.math.RandomUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.io.{LongWritable, Text}
-import org.apache.hadoop.mapred.{FileSplit => OldFileSplit, InputSplit => OldInputSplit, JobConf,
- LineRecordReader => OldLineRecordReader, RecordReader => OldRecordReader, Reporter,
- TextInputFormat => OldTextInputFormat}
import org.apache.hadoop.mapred.lib.{CombineFileInputFormat => OldCombineFileInputFormat,
- CombineFileSplit => OldCombineFileSplit, CombineFileRecordReader => OldCombineFileRecordReader}
-import org.apache.hadoop.mapreduce.{InputSplit => NewInputSplit, RecordReader => NewRecordReader,
- TaskAttemptContext}
+ CombineFileRecordReader => OldCombineFileRecordReader, CombineFileSplit => OldCombineFileSplit}
+import org.apache.hadoop.mapred.{JobConf, Reporter, FileSplit => OldFileSplit,
+ InputSplit => OldInputSplit, LineRecordReader => OldLineRecordReader,
+ RecordReader => OldRecordReader, TextInputFormat => OldTextInputFormat}
import org.apache.hadoop.mapreduce.lib.input.{CombineFileInputFormat => NewCombineFileInputFormat,
CombineFileRecordReader => NewCombineFileRecordReader, CombineFileSplit => NewCombineFileSplit,
FileSplit => NewFileSplit, TextInputFormat => NewTextInputFormat}
+import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat}
+import org.apache.hadoop.mapreduce.{TaskAttemptContext, InputSplit => NewInputSplit,
+ RecordReader => NewRecordReader}
+import org.scalatest.{BeforeAndAfter, FunSuite}
import org.apache.spark.SharedSparkContext
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd}
import org.apache.spark.util.Utils
-class InputOutputMetricsSuite extends FunSuite with SharedSparkContext {
+class InputOutputMetricsSuite extends FunSuite with SharedSparkContext
+ with BeforeAndAfter {
@transient var tmpDir: File = _
@transient var tmpFile: File = _
@transient var tmpFilePath: String = _
+ @transient val numRecords: Int = 100000
+ @transient val numBuckets: Int = 10
- override def beforeAll() {
- super.beforeAll()
-
+ before {
tmpDir = Utils.createTempDir()
val testTempDir = new File(tmpDir, "test")
testTempDir.mkdir()
tmpFile = new File(testTempDir, getClass.getSimpleName + ".txt")
val pw = new PrintWriter(new FileWriter(tmpFile))
- for (x <- 1 to 1000000) {
- pw.println("s")
+ for (x <- 1 to numRecords) {
+ pw.println(RandomUtils.nextInt(numBuckets))
}
pw.close()
@@ -66,8 +68,7 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext {
tmpFilePath = "file://" + tmpFile.getAbsolutePath
}
- override def afterAll() {
- super.afterAll()
+ after {
Utils.deleteRecursively(tmpDir)
}
@@ -155,6 +156,101 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext {
assert(bytesRead >= tmpFile.length())
}
+ test("input metrics on records read - simple") {
+ val records = runAndReturnRecordsRead {
+ sc.textFile(tmpFilePath, 4).count()
+ }
+ assert(records == numRecords)
+ }
+
+ test("input metrics on records read - more stages") {
+ val records = runAndReturnRecordsRead {
+ sc.textFile(tmpFilePath, 4)
+ .map(key => (key.length, 1))
+ .reduceByKey(_ + _)
+ .count()
+ }
+ assert(records == numRecords)
+ }
+
+ test("input metrics on records - New Hadoop API") {
+ val records = runAndReturnRecordsRead {
+ sc.newAPIHadoopFile(tmpFilePath, classOf[NewTextInputFormat], classOf[LongWritable],
+ classOf[Text]).count()
+ }
+ assert(records == numRecords)
+ }
+
+ test("input metrics on recordsd read with cache") {
+ // prime the cache manager
+ val rdd = sc.textFile(tmpFilePath, 4).cache()
+ rdd.collect()
+
+ val records = runAndReturnRecordsRead {
+ rdd.count()
+ }
+
+ assert(records == numRecords)
+ }
+
+ test("shuffle records read metrics") {
+ val recordsRead = runAndReturnShuffleRecordsRead {
+ sc.textFile(tmpFilePath, 4)
+ .map(key => (key, 1))
+ .groupByKey()
+ .collect()
+ }
+ assert(recordsRead == numRecords)
+ }
+
+ test("shuffle records written metrics") {
+ val recordsWritten = runAndReturnShuffleRecordsWritten {
+ sc.textFile(tmpFilePath, 4)
+ .map(key => (key, 1))
+ .groupByKey()
+ .collect()
+ }
+ assert(recordsWritten == numRecords)
+ }
+
+ /**
+ * Tests the metrics from end to end.
+ * 1) reading a hadoop file
+ * 2) shuffle and writing to a hadoop file.
+ * 3) writing to hadoop file.
+ */
+ test("input read/write and shuffle read/write metrics all line up") {
+ var inputRead = 0L
+ var outputWritten = 0L
+ var shuffleRead = 0L
+ var shuffleWritten = 0L
+ sc.addSparkListener(new SparkListener() {
+ override def onTaskEnd(taskEnd: SparkListenerTaskEnd) {
+ val metrics = taskEnd.taskMetrics
+ metrics.inputMetrics.foreach(inputRead += _.recordsRead)
+ metrics.outputMetrics.foreach(outputWritten += _.recordsWritten)
+ metrics.shuffleReadMetrics.foreach(shuffleRead += _.recordsRead)
+ metrics.shuffleWriteMetrics.foreach(shuffleWritten += _.shuffleRecordsWritten)
+ }
+ })
+
+ val tmpFile = new File(tmpDir, getClass.getSimpleName)
+
+ sc.textFile(tmpFilePath, 4)
+ .map(key => (key, 1))
+ .reduceByKey(_+_)
+ .saveAsTextFile("file://" + tmpFile.getAbsolutePath)
+
+ sc.listenerBus.waitUntilEmpty(500)
+ assert(inputRead == numRecords)
+
+ // Only supported on newer Hadoop
+ if (SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback().isDefined) {
+ assert(outputWritten == numBuckets)
+ }
+ assert(shuffleRead == shuffleWritten)
+ }
+
test("input metrics with interleaved reads") {
val numPartitions = 2
val cartVector = 0 to 9
@@ -193,18 +289,66 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext {
assert(cartesianBytes == firstSize * numPartitions + (cartVector.length * secondSize))
}
- private def runAndReturnBytesRead(job : => Unit): Long = {
- val taskBytesRead = new ArrayBuffer[Long]()
+ private def runAndReturnBytesRead(job: => Unit): Long = {
+ runAndReturnMetrics(job, _.taskMetrics.inputMetrics.map(_.bytesRead))
+ }
+
+ private def runAndReturnRecordsRead(job: => Unit): Long = {
+ runAndReturnMetrics(job, _.taskMetrics.inputMetrics.map(_.recordsRead))
+ }
+
+ private def runAndReturnRecordsWritten(job: => Unit): Long = {
+ runAndReturnMetrics(job, _.taskMetrics.outputMetrics.map(_.recordsWritten))
+ }
+
+ private def runAndReturnShuffleRecordsRead(job: => Unit): Long = {
+ runAndReturnMetrics(job, _.taskMetrics.shuffleReadMetrics.map(_.recordsRead))
+ }
+
+ private def runAndReturnShuffleRecordsWritten(job: => Unit): Long = {
+ runAndReturnMetrics(job, _.taskMetrics.shuffleWriteMetrics.map(_.shuffleRecordsWritten))
+ }
+
+ private def runAndReturnMetrics(job: => Unit,
+ collector: (SparkListenerTaskEnd) => Option[Long]): Long = {
+ val taskMetrics = new ArrayBuffer[Long]()
sc.addSparkListener(new SparkListener() {
override def onTaskEnd(taskEnd: SparkListenerTaskEnd) {
- taskBytesRead += taskEnd.taskMetrics.inputMetrics.get.bytesRead
+ collector(taskEnd).foreach(taskMetrics += _)
}
})
job
sc.listenerBus.waitUntilEmpty(500)
- taskBytesRead.sum
+ taskMetrics.sum
+ }
+
+ test("output metrics on records written") {
+ // Only supported on newer Hadoop
+ if (SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback().isDefined) {
+ val file = new File(tmpDir, getClass.getSimpleName)
+ val filePath = "file://" + file.getAbsolutePath
+
+ val records = runAndReturnRecordsWritten {
+ sc.parallelize(1 to numRecords).saveAsTextFile(filePath)
+ }
+ assert(records == numRecords)
+ }
+ }
+
+ test("output metrics on records written - new Hadoop API") {
+ // Only supported on newer Hadoop
+ if (SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback().isDefined) {
+ val file = new File(tmpDir, getClass.getSimpleName)
+ val filePath = "file://" + file.getAbsolutePath
+
+ val records = runAndReturnRecordsWritten {
+ sc.parallelize(1 to numRecords).map(key => (key.toString, key.toString))
+ .saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](filePath)
+ }
+ assert(records == numRecords)
+ }
}
test("output metrics when writing text file") {
@@ -318,4 +462,4 @@ class NewCombineTextRecordReaderWrapper(
override def getCurrentValue(): Text = delegate.getCurrentValue
override def getProgress(): Float = delegate.getProgress
override def close(): Unit = delegate.close()
-}
\ No newline at end of file
+}
diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala
index 1a9ce8c607dcd..37e528435aa5d 100644
--- a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala
+++ b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala
@@ -27,7 +27,7 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter {
}
test("MetricsConfig with default properties") {
- val conf = new MetricsConfig(Option("dummy-file"))
+ val conf = new MetricsConfig(None)
conf.initialize()
assert(conf.properties.size() === 4)
diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala
index de306533752c1..4cd0f97368ca3 100644
--- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala
@@ -33,6 +33,9 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext {
val expectedHistogramResults = Array(0)
assert(histogramResults === expectedHistogramResults)
assert(histogramResults2 === expectedHistogramResults)
+ val emptyRDD: RDD[Double] = sc.emptyRDD
+ assert(emptyRDD.histogram(buckets) === expectedHistogramResults)
+ assert(emptyRDD.histogram(buckets, true) === expectedHistogramResults)
}
test("WorksWithOutOfRangeWithOneBucket") {
diff --git a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala
index 6138d0bbd57f6..0dc59888f7304 100644
--- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala
@@ -29,22 +29,42 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext {
Class.forName("org.apache.derby.jdbc.EmbeddedDriver")
val conn = DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb;create=true")
try {
- val create = conn.createStatement
- create.execute("""
- CREATE TABLE FOO(
- ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1),
- DATA INTEGER
- )""")
- create.close()
- val insert = conn.prepareStatement("INSERT INTO FOO(DATA) VALUES(?)")
- (1 to 100).foreach { i =>
- insert.setInt(1, i * 2)
- insert.executeUpdate
+
+ try {
+ val create = conn.createStatement
+ create.execute("""
+ CREATE TABLE FOO(
+ ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1),
+ DATA INTEGER
+ )""")
+ create.close()
+ val insert = conn.prepareStatement("INSERT INTO FOO(DATA) VALUES(?)")
+ (1 to 100).foreach { i =>
+ insert.setInt(1, i * 2)
+ insert.executeUpdate
+ }
+ insert.close()
+ } catch {
+ case e: SQLException if e.getSQLState == "X0Y32" =>
+ // table exists
}
- insert.close()
- } catch {
- case e: SQLException if e.getSQLState == "X0Y32" =>
+
+ try {
+ val create = conn.createStatement
+ create.execute("CREATE TABLE BIGINT_TEST(ID BIGINT NOT NULL, DATA INTEGER)")
+ create.close()
+ val insert = conn.prepareStatement("INSERT INTO BIGINT_TEST VALUES(?,?)")
+ (1 to 100).foreach { i =>
+ insert.setLong(1, 100000000000000000L + 4000000000000000L * i)
+ insert.setInt(2, i)
+ insert.executeUpdate
+ }
+ insert.close()
+ } catch {
+ case e: SQLException if e.getSQLState == "X0Y32" =>
// table exists
+ }
+
} finally {
conn.close()
}
@@ -62,6 +82,18 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext {
assert(rdd.count === 100)
assert(rdd.reduce(_+_) === 10100)
}
+
+ test("large id overflow") {
+ sc = new SparkContext("local", "test")
+ val rdd = new JdbcRDD(
+ sc,
+ () => { DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb") },
+ "SELECT DATA FROM BIGINT_TEST WHERE ? <= ID AND ID <= ?",
+ 1131544775L, 567279358897692673L, 20,
+ (r: ResultSet) => { r.getInt(1) } ).cache()
+ assert(rdd.count === 100)
+ assert(rdd.reduce(_+_) === 5050)
+ }
after {
try {
diff --git a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala
index a40f2ffeffdf9..64b1c24c47168 100644
--- a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala
@@ -119,5 +119,33 @@ class SortingSuite extends FunSuite with SharedSparkContext with Matchers with L
partitions(1).last should be > partitions(2).head
partitions(2).last should be > partitions(3).head
}
+
+ test("get a range of elements in a sorted RDD that is on one partition") {
+ val pairArr = (1 to 1000).map(x => (x, x)).toArray
+ val sorted = sc.parallelize(pairArr, 10).sortByKey()
+ val range = sorted.filterByRange(20, 40).collect()
+ assert((20 to 40).toArray === range.map(_._1))
+ }
+
+ test("get a range of elements over multiple partitions in a descendingly sorted RDD") {
+ val pairArr = (1000 to 1 by -1).map(x => (x, x)).toArray
+ val sorted = sc.parallelize(pairArr, 10).sortByKey(false)
+ val range = sorted.filterByRange(200, 800).collect()
+ assert((800 to 200 by -1).toArray === range.map(_._1))
+ }
+
+ test("get a range of elements in an array not partitioned by a range partitioner") {
+ val pairArr = util.Random.shuffle((1 to 1000).toList).map(x => (x, x))
+ val pairs = sc.parallelize(pairArr,10)
+ val range = pairs.filterByRange(200, 800).collect()
+ assert((800 to 200 by -1).toArray.sorted === range.map(_._1).sorted)
+ }
+
+ test("get a range of elements over multiple partitions but not taking up full partitions") {
+ val pairArr = (1000 to 1 by -1).map(x => (x, x)).toArray
+ val sorted = sc.parallelize(pairArr, 10).sortByKey(false)
+ val range = sorted.filterByRange(250, 850).collect()
+ assert((850 to 250 by -1).toArray === range.map(_._1))
+ }
}
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 eb116213f69fc..30119ce5d4eec 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -96,6 +96,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar
}
override def setDAGScheduler(dagScheduler: DAGScheduler) = {}
override def defaultParallelism() = 2
+ override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {}
}
/** Length of time to wait while draining listener events. */
@@ -208,7 +209,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar
assert(taskSet.tasks.size >= results.size)
for ((result, i) <- results.zipWithIndex) {
if (i < taskSet.tasks.size) {
- runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, null, null, null))
+ runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, null, createFakeTaskInfo(), null))
}
}
}
@@ -219,7 +220,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar
for ((result, i) <- results.zipWithIndex) {
if (i < taskSet.tasks.size) {
runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2,
- Map[Long, Any]((accumId, 1)), null, null))
+ Map[Long, Any]((accumId, 1)), createFakeTaskInfo(), null))
}
}
}
@@ -386,6 +387,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar
override def defaultParallelism() = 2
override def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)],
blockManagerId: BlockManagerId): Boolean = true
+ override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {}
}
val noKillScheduler = new DAGScheduler(
sc,
@@ -476,7 +478,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar
FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"),
null,
Map[Long, Any](),
- null,
+ createFakeTaskInfo(),
null))
assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
assert(sparkListener.failedStages.contains(1))
@@ -487,7 +489,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar
FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1, "ignored"),
null,
Map[Long, Any](),
- null,
+ createFakeTaskInfo(),
null))
// The SparkListener should not receive redundant failure events.
assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
@@ -507,14 +509,14 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar
assert(newEpoch > oldEpoch)
val taskSet = taskSets(0)
// should be ignored for being too old
- runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, null, null))
+ runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null))
// should work because it's a non-failed host
- runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, null, null))
+ runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null))
// should be ignored for being too old
- runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, null, null))
+ runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null))
// should work because it's a new epoch
taskSet.tasks(1).epoch = newEpoch
- runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, null, null))
+ runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null))
assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) ===
Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA")))
complete(taskSets(1), Seq((Success, 42), (Success, 43)))
@@ -735,7 +737,11 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar
completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42)))
completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42)))
assert(results === Map(0 -> 42))
- assert(Accumulators.originals(accum.id).value === 1)
+
+ val accVal = Accumulators.originals(accum.id).get.get.value
+
+ assert(accVal === 1)
+
assertDataStructuresEmpty
}
@@ -766,5 +772,14 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar
assert(scheduler.shuffleToMapStage.isEmpty)
assert(scheduler.waitingStages.isEmpty)
}
+
+ // Nothing in this test should break if the task info's fields are null, but
+ // OutputCommitCoordinator requires the task info itself to not be null.
+ private def createFakeTaskInfo(): TaskInfo = {
+ val info = new TaskInfo(0, 0, 0, 0L, "", "", TaskLocality.ANY, false)
+ info.finishTime = 1 // to prevent spurious errors in JobProgressListener
+ info
+ }
+
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
index 437d8693c0b1f..992dde66f982f 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
@@ -18,6 +18,7 @@
package org.apache.spark.scheduler
import java.io.{File, FileOutputStream, InputStream, IOException}
+import java.net.URI
import scala.collection.mutable
import scala.io.Source
@@ -26,7 +27,7 @@ import org.apache.hadoop.fs.Path
import org.json4s.jackson.JsonMethods._
import org.scalatest.{BeforeAndAfter, FunSuite}
-import org.apache.spark.{Logging, SparkConf, SparkContext}
+import org.apache.spark.{Logging, SparkConf, SparkContext, SPARK_VERSION}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.io._
import org.apache.spark.util.{JsonProtocol, Utils}
@@ -78,7 +79,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin
test("Basic event logging with compression") {
CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec =>
- testEventLogging(compressionCodec = Some(codec))
+ testEventLogging(compressionCodec = Some(CompressionCodec.getShortName(codec)))
}
}
@@ -88,25 +89,35 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin
test("End-to-end event logging with compression") {
CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec =>
- testApplicationEventLogging(compressionCodec = Some(codec))
+ testApplicationEventLogging(compressionCodec = Some(CompressionCodec.getShortName(codec)))
}
}
test("Log overwriting") {
- val log = new FileOutputStream(new File(testDir, "test"))
- log.close()
- try {
- testEventLogging()
- assert(false)
- } catch {
- case e: IOException =>
- // Expected, since we haven't enabled log overwrite.
- }
-
+ val logUri = EventLoggingListener.getLogPath(testDir.getAbsolutePath, "test")
+ val logPath = new URI(logUri).getPath
+ // Create file before writing the event log
+ new FileOutputStream(new File(logPath)).close()
+ // Expected IOException, since we haven't enabled log overwrite.
+ intercept[IOException] { testEventLogging() }
// Try again, but enable overwriting.
testEventLogging(extraConf = Map("spark.eventLog.overwrite" -> "true"))
}
+ test("Event log name") {
+ // without compression
+ assert(s"file:/base-dir/app1" === EventLoggingListener.getLogPath("/base-dir", "app1"))
+ // with compression
+ assert(s"file:/base-dir/app1.lzf" ===
+ EventLoggingListener.getLogPath("/base-dir", "app1", Some("lzf")))
+ // illegal characters in app ID
+ assert(s"file:/base-dir/a-fine-mind_dollar_bills__1" ===
+ EventLoggingListener.getLogPath("/base-dir", "a fine:mind$dollar{bills}.1"))
+ // illegal characters in app ID with compression
+ assert(s"file:/base-dir/a-fine-mind_dollar_bills__1.lz4" ===
+ EventLoggingListener.getLogPath("/base-dir", "a fine:mind$dollar{bills}.1", Some("lz4")))
+ }
+
/* ----------------- *
* Actual test logic *
* ----------------- */
@@ -140,15 +151,17 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin
eventLogger.stop()
// Verify file contains exactly the two events logged
- val (logData, version) = EventLoggingListener.openEventLog(new Path(eventLogger.logPath),
- fileSystem)
+ val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem)
try {
val lines = readLines(logData)
- assert(lines.size === 2)
- assert(lines(0).contains("SparkListenerApplicationStart"))
- assert(lines(1).contains("SparkListenerApplicationEnd"))
- assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === applicationStart)
- assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationEnd)
+ val logStart = SparkListenerLogStart(SPARK_VERSION)
+ assert(lines.size === 3)
+ assert(lines(0).contains("SparkListenerLogStart"))
+ assert(lines(1).contains("SparkListenerApplicationStart"))
+ assert(lines(2).contains("SparkListenerApplicationEnd"))
+ assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart)
+ assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationStart)
+ assert(JsonProtocol.sparkEventFromJson(parse(lines(2))) === applicationEnd)
} finally {
logData.close()
}
@@ -163,8 +176,10 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin
val sc = new SparkContext("local-cluster[2,2,512]", "test", conf)
assert(sc.eventLogger.isDefined)
val eventLogger = sc.eventLogger.get
+ val eventLogPath = eventLogger.logPath
val expectedLogDir = testDir.toURI().toString()
- assert(eventLogger.logPath.startsWith(expectedLogDir + "/"))
+ assert(eventLogPath === EventLoggingListener.getLogPath(
+ expectedLogDir, sc.applicationId, compressionCodec.map(CompressionCodec.getShortName)))
// Begin listening for events that trigger asserts
val eventExistenceListener = new EventExistenceListener(eventLogger)
@@ -178,8 +193,8 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin
eventExistenceListener.assertAllCallbacksInvoked()
// Make sure expected events exist in the log file.
- val (logData, version) = EventLoggingListener.openEventLog(new Path(eventLogger.logPath),
- fileSystem)
+ val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem)
+ val logStart = SparkListenerLogStart(SPARK_VERSION)
val lines = readLines(logData)
val eventSet = mutable.Set(
SparkListenerApplicationStart,
@@ -204,6 +219,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin
}
}
}
+ assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart)
assert(eventSet.isEmpty, "The following events are missing: " + eventSet.toSeq)
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
new file mode 100644
index 0000000000000..c8c957856247a
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
@@ -0,0 +1,213 @@
+/*
+ * 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.scheduler
+
+import java.io.File
+import java.util.concurrent.TimeoutException
+
+import org.mockito.Matchers
+import org.mockito.Mockito._
+import org.mockito.invocation.InvocationOnMock
+import org.mockito.stubbing.Answer
+import org.scalatest.{BeforeAndAfter, FunSuite}
+
+import org.apache.hadoop.mapred.{TaskAttemptID, JobConf, TaskAttemptContext, OutputCommitter}
+
+import org.apache.spark._
+import org.apache.spark.rdd.{RDD, FakeOutputCommitter}
+import org.apache.spark.util.Utils
+
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+/**
+ * Unit tests for the output commit coordination functionality.
+ *
+ * The unit test makes both the original task and the speculated task
+ * attempt to commit, where committing is emulated by creating a
+ * directory. If both tasks create directories then the end result is
+ * a failure.
+ *
+ * Note that there are some aspects of this test that are less than ideal.
+ * In particular, the test mocks the speculation-dequeuing logic to always
+ * dequeue a task and consider it as speculated. Immediately after initially
+ * submitting the tasks and calling reviveOffers(), reviveOffers() is invoked
+ * again to pick up the speculated task. This may be hacking the original
+ * behavior in too much of an unrealistic fashion.
+ *
+ * Also, the validation is done by checking the number of files in a directory.
+ * Ideally, an accumulator would be used for this, where we could increment
+ * the accumulator in the output committer's commitTask() call. If the call to
+ * commitTask() was called twice erroneously then the test would ideally fail because
+ * the accumulator would be incremented twice.
+ *
+ * The problem with this test implementation is that when both a speculated task and
+ * its original counterpart complete, only one of the accumulator's increments is
+ * captured. This results in a paradox where if the OutputCommitCoordinator logic
+ * was not in SparkHadoopWriter, the tests would still pass because only one of the
+ * increments would be captured even though the commit in both tasks was executed
+ * erroneously.
+ */
+class OutputCommitCoordinatorSuite extends FunSuite with BeforeAndAfter {
+
+ var outputCommitCoordinator: OutputCommitCoordinator = null
+ var tempDir: File = null
+ var sc: SparkContext = null
+
+ before {
+ tempDir = Utils.createTempDir()
+ val conf = new SparkConf()
+ .setMaster("local[4]")
+ .setAppName(classOf[OutputCommitCoordinatorSuite].getSimpleName)
+ .set("spark.speculation", "true")
+ sc = new SparkContext(conf) {
+ override private[spark] def createSparkEnv(
+ conf: SparkConf,
+ isLocal: Boolean,
+ listenerBus: LiveListenerBus): SparkEnv = {
+ outputCommitCoordinator = spy(new OutputCommitCoordinator(conf))
+ // Use Mockito.spy() to maintain the default infrastructure everywhere else.
+ // This mocking allows us to control the coordinator responses in test cases.
+ SparkEnv.createDriverEnv(conf, isLocal, listenerBus, Some(outputCommitCoordinator))
+ }
+ }
+ // Use Mockito.spy() to maintain the default infrastructure everywhere else
+ val mockTaskScheduler = spy(sc.taskScheduler.asInstanceOf[TaskSchedulerImpl])
+
+ doAnswer(new Answer[Unit]() {
+ override def answer(invoke: InvocationOnMock): Unit = {
+ // Submit the tasks, then force the task scheduler to dequeue the
+ // speculated task
+ invoke.callRealMethod()
+ mockTaskScheduler.backend.reviveOffers()
+ }
+ }).when(mockTaskScheduler).submitTasks(Matchers.any())
+
+ doAnswer(new Answer[TaskSetManager]() {
+ override def answer(invoke: InvocationOnMock): TaskSetManager = {
+ val taskSet = invoke.getArguments()(0).asInstanceOf[TaskSet]
+ new TaskSetManager(mockTaskScheduler, taskSet, 4) {
+ var hasDequeuedSpeculatedTask = false
+ override def dequeueSpeculativeTask(
+ execId: String,
+ host: String,
+ locality: TaskLocality.Value): Option[(Int, TaskLocality.Value)] = {
+ if (!hasDequeuedSpeculatedTask) {
+ hasDequeuedSpeculatedTask = true
+ Some(0, TaskLocality.PROCESS_LOCAL)
+ } else {
+ None
+ }
+ }
+ }
+ }
+ }).when(mockTaskScheduler).createTaskSetManager(Matchers.any(), Matchers.any())
+
+ sc.taskScheduler = mockTaskScheduler
+ val dagSchedulerWithMockTaskScheduler = new DAGScheduler(sc, mockTaskScheduler)
+ sc.taskScheduler.setDAGScheduler(dagSchedulerWithMockTaskScheduler)
+ sc.dagScheduler = dagSchedulerWithMockTaskScheduler
+ }
+
+ after {
+ sc.stop()
+ tempDir.delete()
+ outputCommitCoordinator = null
+ }
+
+ test("Only one of two duplicate commit tasks should commit") {
+ val rdd = sc.parallelize(Seq(1), 1)
+ sc.runJob(rdd, OutputCommitFunctions(tempDir.getAbsolutePath).commitSuccessfully _,
+ 0 until rdd.partitions.size, allowLocal = false)
+ assert(tempDir.list().size === 1)
+ }
+
+ test("If commit fails, if task is retried it should not be locked, and will succeed.") {
+ val rdd = sc.parallelize(Seq(1), 1)
+ sc.runJob(rdd, OutputCommitFunctions(tempDir.getAbsolutePath).failFirstCommitAttempt _,
+ 0 until rdd.partitions.size, allowLocal = false)
+ assert(tempDir.list().size === 1)
+ }
+
+ test("Job should not complete if all commits are denied") {
+ // Create a mock OutputCommitCoordinator that denies all attempts to commit
+ doReturn(false).when(outputCommitCoordinator).handleAskPermissionToCommit(
+ Matchers.any(), Matchers.any(), Matchers.any())
+ val rdd: RDD[Int] = sc.parallelize(Seq(1), 1)
+ def resultHandler(x: Int, y: Unit): Unit = {}
+ val futureAction: SimpleFutureAction[Unit] = sc.submitJob[Int, Unit, Unit](rdd,
+ OutputCommitFunctions(tempDir.getAbsolutePath).commitSuccessfully,
+ 0 until rdd.partitions.size, resultHandler, () => Unit)
+ // It's an error if the job completes successfully even though no committer was authorized,
+ // so throw an exception if the job was allowed to complete.
+ intercept[TimeoutException] {
+ Await.result(futureAction, 5 seconds)
+ }
+ assert(tempDir.list().size === 0)
+ }
+}
+
+/**
+ * Class with methods that can be passed to runJob to test commits with a mock committer.
+ */
+private case class OutputCommitFunctions(tempDirPath: String) {
+
+ // Mock output committer that simulates a successful commit (after commit is authorized)
+ private def successfulOutputCommitter = new FakeOutputCommitter {
+ override def commitTask(context: TaskAttemptContext): Unit = {
+ Utils.createDirectory(tempDirPath)
+ }
+ }
+
+ // Mock output committer that simulates a failed commit (after commit is authorized)
+ private def failingOutputCommitter = new FakeOutputCommitter {
+ override def commitTask(taskAttemptContext: TaskAttemptContext) {
+ throw new RuntimeException
+ }
+ }
+
+ def commitSuccessfully(iter: Iterator[Int]): Unit = {
+ val ctx = TaskContext.get()
+ runCommitWithProvidedCommitter(ctx, iter, successfulOutputCommitter)
+ }
+
+ def failFirstCommitAttempt(iter: Iterator[Int]): Unit = {
+ val ctx = TaskContext.get()
+ runCommitWithProvidedCommitter(ctx, iter,
+ if (ctx.attemptNumber == 0) failingOutputCommitter else successfulOutputCommitter)
+ }
+
+ private def runCommitWithProvidedCommitter(
+ ctx: TaskContext,
+ iter: Iterator[Int],
+ outputCommitter: OutputCommitter): Unit = {
+ def jobConf = new JobConf {
+ override def getOutputCommitter(): OutputCommitter = outputCommitter
+ }
+ val sparkHadoopWriter = new SparkHadoopWriter(jobConf) {
+ override def newTaskAttemptContext(
+ conf: JobConf,
+ attemptId: TaskAttemptID): TaskAttemptContext = {
+ mock(classOf[TaskAttemptContext])
+ }
+ }
+ sparkHadoopWriter.setup(ctx.stageId, ctx.partitionId, ctx.attemptNumber)
+ sparkHadoopWriter.commit()
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
index 7e360cc6082ec..601694f57aad0 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
@@ -61,7 +61,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter {
try {
val replayer = new ReplayListenerBus()
replayer.addListener(eventMonster)
- replayer.replay(logData, SPARK_VERSION)
+ replayer.replay(logData, logFilePath.toString)
} finally {
logData.close()
}
@@ -115,12 +115,12 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter {
assert(!eventLog.isDir)
// Replay events
- val (logData, version) = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem)
+ val logData = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem)
val eventMonster = new EventMonster(conf)
try {
val replayer = new ReplayListenerBus()
replayer.addListener(eventMonster)
- replayer.replay(logData, version)
+ replayer.replay(logData, eventLog.getPath().toString)
} finally {
logData.close()
}
@@ -150,11 +150,4 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter {
override def start() { }
}
-
- private def getCompressionCodec(codecName: String) = {
- val conf = new SparkConf
- conf.set("spark.io.compression.codec", codecName)
- CompressionCodec.createCodec(conf)
- }
-
}
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 0fb1bdd30d975..3a41ee8d4ae0c 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -20,26 +20,22 @@ package org.apache.spark.scheduler
import java.util.concurrent.Semaphore
import scala.collection.mutable
+import scala.collection.JavaConversions._
-import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite}
-import org.scalatest.Matchers
+import org.scalatest.{FunSuite, Matchers}
-import org.apache.spark.{LocalSparkContext, SparkContext}
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.util.ResetSystemProperties
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext}
-class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers with BeforeAndAfter
- with BeforeAndAfterAll with ResetSystemProperties {
+class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers
+ with ResetSystemProperties {
/** Length of time to wait while draining listener events. */
val WAIT_TIMEOUT_MILLIS = 10000
val jobCompletionTime = 1421191296660L
- before {
- sc = new SparkContext("local", "SparkListenerSuite")
- }
-
test("basic creation and shutdown of LiveListenerBus") {
val counter = new BasicJobCounter
val bus = new LiveListenerBus
@@ -127,6 +123,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers
}
test("basic creation of StageInfo") {
+ sc = new SparkContext("local", "SparkListenerSuite")
val listener = new SaveStageAndTaskInfo
sc.addSparkListener(listener)
val rdd1 = sc.parallelize(1 to 100, 4)
@@ -148,6 +145,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers
}
test("basic creation of StageInfo with shuffle") {
+ sc = new SparkContext("local", "SparkListenerSuite")
val listener = new SaveStageAndTaskInfo
sc.addSparkListener(listener)
val rdd1 = sc.parallelize(1 to 100, 4)
@@ -185,6 +183,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers
}
test("StageInfo with fewer tasks than partitions") {
+ sc = new SparkContext("local", "SparkListenerSuite")
val listener = new SaveStageAndTaskInfo
sc.addSparkListener(listener)
val rdd1 = sc.parallelize(1 to 100, 4)
@@ -201,6 +200,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers
}
test("local metrics") {
+ sc = new SparkContext("local", "SparkListenerSuite")
val listener = new SaveStageAndTaskInfo
sc.addSparkListener(listener)
sc.addSparkListener(new StatsReportListener)
@@ -267,6 +267,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers
}
test("onTaskGettingResult() called when result fetched remotely") {
+ sc = new SparkContext("local", "SparkListenerSuite")
val listener = new SaveTaskEvents
sc.addSparkListener(listener)
@@ -287,6 +288,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers
}
test("onTaskGettingResult() not called when result sent directly") {
+ sc = new SparkContext("local", "SparkListenerSuite")
val listener = new SaveTaskEvents
sc.addSparkListener(listener)
@@ -302,6 +304,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers
}
test("onTaskEnd() should be called for all started tasks, even after job has been killed") {
+ sc = new SparkContext("local", "SparkListenerSuite")
val WAIT_TIMEOUT_MILLIS = 10000
val listener = new SaveTaskEvents
sc.addSparkListener(listener)
@@ -356,6 +359,17 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers
assert(jobCounter2.count === 5)
}
+ test("registering listeners via spark.extraListeners") {
+ val conf = new SparkConf().setMaster("local").setAppName("test")
+ .set("spark.extraListeners", classOf[ListenerThatAcceptsSparkConf].getName + "," +
+ classOf[BasicJobCounter].getName)
+ sc = new SparkContext(conf)
+ sc.listenerBus.listeners.collect { case x: BasicJobCounter => x}.size should be (1)
+ sc.listenerBus.listeners.collect {
+ case x: ListenerThatAcceptsSparkConf => x
+ }.size should be (1)
+ }
+
/**
* Assert that the given list of numbers has an average that is greater than zero.
*/
@@ -363,14 +377,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers
assert(m.sum / m.size.toDouble > 0.0, msg)
}
- /**
- * A simple listener that counts the number of jobs observed.
- */
- private class BasicJobCounter extends SparkListener {
- var count = 0
- override def onJobEnd(job: SparkListenerJobEnd) = count += 1
- }
-
/**
* A simple listener that saves all task infos and task metrics.
*/
@@ -423,3 +429,19 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers
}
}
+
+// These classes can't be declared inside of the SparkListenerSuite class because we don't want
+// their constructors to contain references to SparkListenerSuite:
+
+/**
+ * A simple listener that counts the number of jobs observed.
+ */
+private class BasicJobCounter extends SparkListener {
+ var count = 0
+ override def onJobEnd(job: SparkListenerJobEnd) = count += 1
+}
+
+private class ListenerThatAcceptsSparkConf(conf: SparkConf) extends SparkListener {
+ var count = 0
+ override def onJobEnd(job: SparkListenerJobEnd) = count += 1
+}
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 84b9b788237bf..12330d8f63c40 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
@@ -27,7 +27,7 @@ import org.scalatest.FunSuite
import org.apache.spark._
import org.apache.spark.executor.TaskMetrics
-import org.apache.spark.util.FakeClock
+import org.apache.spark.util.ManualClock
class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler)
extends DAGScheduler(sc) {
@@ -164,7 +164,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
sc = new SparkContext("local", "test")
val sched = new FakeTaskScheduler(sc, ("exec1", "host1"))
val taskSet = FakeTask.createTaskSet(1)
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// Offer a host with NO_PREF as the constraint,
@@ -213,7 +213,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
sc = new SparkContext("local", "test")
val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execC", "host2"))
val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "execB")))
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// An executor that is not NODE_LOCAL should be rejected.
@@ -234,7 +234,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
Seq(TaskLocation("host1"), TaskLocation("host2", "exec2")),
Seq() // Last task has no locality prefs
)
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// First offer host1, exec1: first task should be chosen
assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0)
@@ -263,7 +263,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
Seq(TaskLocation("host2", "exec3")),
Seq() // Last task has no locality prefs
)
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// First offer host1, exec1: first task should be chosen
assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).get.index === 0)
@@ -283,7 +283,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
Seq(TaskLocation("host3")),
Seq(TaskLocation("host2"))
)
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// First offer host1: first task should be chosen
@@ -314,13 +314,14 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
test("delay scheduling with failed hosts") {
sc = new SparkContext("local", "test")
- val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"))
+ val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"),
+ ("exec3", "host3"))
val taskSet = FakeTask.createTaskSet(3,
Seq(TaskLocation("host1")),
Seq(TaskLocation("host2")),
Seq(TaskLocation("host3"))
)
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// First offer host1: first task should be chosen
@@ -352,7 +353,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
sc = new SparkContext("local", "test")
val sched = new FakeTaskScheduler(sc, ("exec1", "host1"))
val taskSet = FakeTask.createTaskSet(1)
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0)
@@ -369,7 +370,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
sc = new SparkContext("local", "test")
val sched = new FakeTaskScheduler(sc, ("exec1", "host1"))
val taskSet = FakeTask.createTaskSet(1)
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted
@@ -401,7 +402,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
("exec1.1", "host1"), ("exec2", "host2"))
// affinity to exec1 on host1 - which we will fail.
val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "exec1")))
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, 4, clock)
{
@@ -485,7 +486,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
Seq(TaskLocation("host1", "execB")),
Seq(TaskLocation("host2", "execC")),
Seq())
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// Only ANY is valid
assert(manager.myLocalityLevels.sameElements(Array(NO_PREF, ANY)))
@@ -521,7 +522,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
val taskSet = FakeTask.createTaskSet(2,
Seq(TaskLocation("host1", "execA")),
Seq(TaskLocation("host1", "execA")))
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY)))
@@ -610,7 +611,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
Seq(TaskLocation("host2"), TaskLocation("host1")),
Seq(),
Seq(TaskLocation("host3", "execC")))
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 0)
@@ -636,7 +637,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
Seq(TaskLocation("host2")),
Seq(),
Seq(TaskLocation("host3")))
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// node-local tasks are scheduled without delay
@@ -649,6 +650,47 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
assert(manager.resourceOffer("execA", "host3", NO_PREF).get.index === 2)
}
+ test("SPARK-4939: node-local tasks should be scheduled right after process-local tasks finished") {
+ sc = new SparkContext("local", "test")
+ val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"))
+ val taskSet = FakeTask.createTaskSet(4,
+ Seq(TaskLocation("host1")),
+ Seq(TaskLocation("host2")),
+ Seq(ExecutorCacheTaskLocation("host1", "execA")),
+ Seq(ExecutorCacheTaskLocation("host2", "execB")))
+ val clock = new ManualClock
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+
+ // process-local tasks are scheduled first
+ assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 2)
+ assert(manager.resourceOffer("execB", "host2", NODE_LOCAL).get.index === 3)
+ // node-local tasks are scheduled without delay
+ assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0)
+ assert(manager.resourceOffer("execB", "host2", NODE_LOCAL).get.index === 1)
+ assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None)
+ assert(manager.resourceOffer("execB", "host2", NODE_LOCAL) == None)
+ }
+
+ test("SPARK-4939: no-pref tasks should be scheduled after process-local tasks finished") {
+ sc = new SparkContext("local", "test")
+ val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"))
+ val taskSet = FakeTask.createTaskSet(3,
+ Seq(),
+ Seq(ExecutorCacheTaskLocation("host1", "execA")),
+ Seq(ExecutorCacheTaskLocation("host2", "execB")))
+ val clock = new ManualClock
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+
+ // process-local tasks are scheduled first
+ assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 1)
+ assert(manager.resourceOffer("execB", "host2", PROCESS_LOCAL).get.index === 2)
+ // no-pref tasks are scheduled without delay
+ assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL) == None)
+ assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None)
+ assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 0)
+ assert(manager.resourceOffer("execA", "host1", ANY) == None)
+ }
+
test("Ensure TaskSetManager is usable after addition of levels") {
// Regression test for SPARK-2931
sc = new SparkContext("local", "test")
@@ -656,7 +698,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
val taskSet = FakeTask.createTaskSet(2,
Seq(TaskLocation("host1", "execA")),
Seq(TaskLocation("host2", "execB.1")))
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// Only ANY is valid
assert(manager.myLocalityLevels.sameElements(Array(ANY)))
@@ -690,7 +732,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
Seq(HostTaskLocation("host1")),
Seq(HostTaskLocation("host2")),
Seq(HDFSCacheTaskLocation("host3")))
- val clock = new FakeClock
+ val clock = new ManualClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY)))
sched.removeExecutor("execA")
diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala
index f2ff98eb72daf..afbaa9ade811f 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala
@@ -17,45 +17,48 @@
package org.apache.spark.scheduler.mesos
-import org.apache.spark.executor.MesosExecutorBackend
-import org.scalatest.FunSuite
-import org.apache.spark.{SparkConf, SparkContext, LocalSparkContext}
-import org.apache.spark.scheduler.{SparkListenerExecutorAdded, LiveListenerBus,
- TaskDescription, WorkerOffer, TaskSchedulerImpl}
-import org.apache.spark.scheduler.cluster.ExecutorInfo
-import org.apache.spark.scheduler.cluster.mesos.{MemoryUtils, MesosSchedulerBackend}
-import org.apache.mesos.SchedulerDriver
-import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, _}
-import org.apache.mesos.Protos.Value.Scalar
-import org.easymock.{Capture, EasyMock}
import java.nio.ByteBuffer
-import java.util.Collections
import java.util
-import org.scalatest.mock.EasyMockSugar
+import java.util.Collections
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
-class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with EasyMockSugar {
+import org.apache.mesos.SchedulerDriver
+import org.apache.mesos.Protos._
+import org.apache.mesos.Protos.Value.Scalar
+import org.mockito.Mockito._
+import org.mockito.Matchers._
+import org.mockito.{ArgumentCaptor, Matchers}
+import org.scalatest.FunSuite
+import org.scalatest.mock.MockitoSugar
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext}
+import org.apache.spark.executor.MesosExecutorBackend
+import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded,
+ TaskDescription, TaskSchedulerImpl, WorkerOffer}
+import org.apache.spark.scheduler.cluster.ExecutorInfo
+import org.apache.spark.scheduler.cluster.mesos.{MesosSchedulerBackend, MemoryUtils}
+
+class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar {
test("check spark-class location correctly") {
val conf = new SparkConf
conf.set("spark.mesos.executor.home" , "/mesos-home")
- val listenerBus = EasyMock.createMock(classOf[LiveListenerBus])
- listenerBus.post(SparkListenerExecutorAdded(EasyMock.anyLong, "s1", new ExecutorInfo("host1", 2)))
- EasyMock.replay(listenerBus)
-
- val sc = EasyMock.createMock(classOf[SparkContext])
- EasyMock.expect(sc.getSparkHome()).andReturn(Option("/spark-home")).anyTimes()
- EasyMock.expect(sc.conf).andReturn(conf).anyTimes()
- EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes()
- EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes()
- EasyMock.expect(sc.listenerBus).andReturn(listenerBus)
- EasyMock.replay(sc)
- val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl])
- EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes()
- EasyMock.replay(taskScheduler)
+ val listenerBus = mock[LiveListenerBus]
+ listenerBus.post(
+ SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty)))
+
+ val sc = mock[SparkContext]
+ when(sc.getSparkHome()).thenReturn(Option("/spark-home"))
+
+ when(sc.conf).thenReturn(conf)
+ when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String])
+ when(sc.executorMemory).thenReturn(100)
+ when(sc.listenerBus).thenReturn(listenerBus)
+ val taskScheduler = mock[TaskSchedulerImpl]
+ when(taskScheduler.CPUS_PER_TASK).thenReturn(2)
val mesosSchedulerBackend = new MesosSchedulerBackend(taskScheduler, sc, "master")
@@ -84,20 +87,19 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea
.setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")).setHostname(s"host${id.toString}").build()
}
- val driver = EasyMock.createMock(classOf[SchedulerDriver])
- val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl])
+ val driver = mock[SchedulerDriver]
+ val taskScheduler = mock[TaskSchedulerImpl]
- val listenerBus = EasyMock.createMock(classOf[LiveListenerBus])
- listenerBus.post(SparkListenerExecutorAdded(EasyMock.anyLong, "s1", new ExecutorInfo("host1", 2)))
- EasyMock.replay(listenerBus)
+ val listenerBus = mock[LiveListenerBus]
+ listenerBus.post(
+ SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty)))
- val sc = EasyMock.createMock(classOf[SparkContext])
- EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes()
- EasyMock.expect(sc.getSparkHome()).andReturn(Option("/path")).anyTimes()
- EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes()
- EasyMock.expect(sc.conf).andReturn(new SparkConf).anyTimes()
- EasyMock.expect(sc.listenerBus).andReturn(listenerBus)
- EasyMock.replay(sc)
+ val sc = mock[SparkContext]
+ when(sc.executorMemory).thenReturn(100)
+ when(sc.getSparkHome()).thenReturn(Option("/path"))
+ when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String])
+ when(sc.conf).thenReturn(new SparkConf)
+ when(sc.listenerBus).thenReturn(listenerBus)
val minMem = MemoryUtils.calculateTotalMemory(sc).toInt
val minCpu = 4
@@ -121,25 +123,29 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea
2
))
val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0)))
- EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(expectedWorkerOffers))).andReturn(Seq(Seq(taskDesc)))
- EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes()
- EasyMock.replay(taskScheduler)
+ when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc)))
+ when(taskScheduler.CPUS_PER_TASK).thenReturn(2)
- val capture = new Capture[util.Collection[TaskInfo]]
- EasyMock.expect(
+ val capture = ArgumentCaptor.forClass(classOf[util.Collection[TaskInfo]])
+ when(
driver.launchTasks(
- EasyMock.eq(Collections.singleton(mesosOffers.get(0).getId)),
- EasyMock.capture(capture),
- EasyMock.anyObject(classOf[Filters])
+ Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)),
+ capture.capture(),
+ any(classOf[Filters])
)
- ).andReturn(Status.valueOf(1)).once
- EasyMock.expect(driver.declineOffer(mesosOffers.get(1).getId)).andReturn(Status.valueOf(1)).times(1)
- EasyMock.expect(driver.declineOffer(mesosOffers.get(2).getId)).andReturn(Status.valueOf(1)).times(1)
- EasyMock.replay(driver)
+ ).thenReturn(Status.valueOf(1))
+ when(driver.declineOffer(mesosOffers.get(1).getId)).thenReturn(Status.valueOf(1))
+ when(driver.declineOffer(mesosOffers.get(2).getId)).thenReturn(Status.valueOf(1))
backend.resourceOffers(driver, mesosOffers)
- EasyMock.verify(driver)
+ verify(driver, times(1)).launchTasks(
+ Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)),
+ capture.capture(),
+ any(classOf[Filters])
+ )
+ verify(driver, times(1)).declineOffer(mesosOffers.get(1).getId)
+ verify(driver, times(1)).declineOffer(mesosOffers.get(2).getId)
assert(capture.getValue.size() == 1)
val taskInfo = capture.getValue.iterator().next()
assert(taskInfo.getName.equals("n1"))
@@ -151,15 +157,13 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea
// Unwanted resources offered on an existing node. Make sure they are declined
val mesosOffers2 = new java.util.ArrayList[Offer]
mesosOffers2.add(createOffer(1, minMem, minCpu))
- EasyMock.reset(taskScheduler)
- EasyMock.reset(driver)
- EasyMock.expect(taskScheduler.resourceOffers(EasyMock.anyObject(classOf[Seq[WorkerOffer]])).andReturn(Seq(Seq())))
- EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes()
- EasyMock.replay(taskScheduler)
- EasyMock.expect(driver.declineOffer(mesosOffers2.get(0).getId)).andReturn(Status.valueOf(1)).times(1)
- EasyMock.replay(driver)
+ reset(taskScheduler)
+ reset(driver)
+ when(taskScheduler.resourceOffers(any(classOf[Seq[WorkerOffer]]))).thenReturn(Seq(Seq()))
+ when(taskScheduler.CPUS_PER_TASK).thenReturn(2)
+ when(driver.declineOffer(mesosOffers2.get(0).getId)).thenReturn(Status.valueOf(1))
backend.resourceOffers(driver, mesosOffers2)
- EasyMock.verify(driver)
+ verify(driver, times(1)).declineOffer(mesosOffers2.get(0).getId)
}
}
diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala
index 855f1b6276089..054a4c64897a9 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala
@@ -29,9 +29,9 @@ class KryoSerializerDistributedSuite extends FunSuite {
test("kryo objects are serialised consistently in different processes") {
val conf = new SparkConf(false)
- conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
- conf.set("spark.kryo.registrator", classOf[AppJarRegistrator].getName)
- conf.set("spark.task.maxFailures", "1")
+ .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+ .set("spark.kryo.registrator", classOf[AppJarRegistrator].getName)
+ .set("spark.task.maxFailures", "1")
val jar = TestUtils.createJarWithClasses(List(AppJarRegistrator.customClassName))
conf.setJars(List(jar.getPath))
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 a70f67af2e62e..6198df84fab3d 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
@@ -23,9 +23,10 @@ import scala.reflect.ClassTag
import com.esotericsoftware.kryo.Kryo
import org.scalatest.FunSuite
-import org.apache.spark.{SparkConf, SharedSparkContext}
+import org.apache.spark.{SharedSparkContext, SparkConf}
+import org.apache.spark.scheduler.HighlyCompressedMapStatus
import org.apache.spark.serializer.KryoTest._
-
+import org.apache.spark.storage.BlockManagerId
class KryoSerializerSuite extends FunSuite with SharedSparkContext {
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
@@ -242,6 +243,38 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
ser.newInstance().deserialize[ClassLoaderTestingObject](bytes)
}
}
+
+ test("registration of HighlyCompressedMapStatus") {
+ val conf = new SparkConf(false)
+ conf.set("spark.kryo.registrationRequired", "true")
+
+ // these cases require knowing the internals of RoaringBitmap a little. Blocks span 2^16
+ // values, and they use a bitmap (dense) if they have more than 4096 values, and an
+ // array (sparse) if they use less. So we just create two cases, one sparse and one dense.
+ // and we use a roaring bitmap for the empty blocks, so we trigger the dense case w/ mostly
+ // empty blocks
+
+ val ser = new KryoSerializer(conf).newInstance()
+ val denseBlockSizes = new Array[Long](5000)
+ val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L)
+ Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes =>
+ ser.serialize(HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), blockSizes))
+ }
+ }
+
+ test("serialization buffer overflow reporting") {
+ import org.apache.spark.SparkException
+ val kryoBufferMaxProperty = "spark.kryoserializer.buffer.max.mb"
+
+ val largeObject = (1 to 1000000).toArray
+
+ val conf = new SparkConf(false)
+ conf.set(kryoBufferMaxProperty, "1")
+
+ val ser = new KryoSerializer(conf).newInstance()
+ val thrown = intercept[SparkException](ser.serialize(largeObject))
+ assert(thrown.getMessage.contains(kryoBufferMaxProperty))
+ }
}
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 ffe6f039145ea..3fdbe99b5d02b 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -1064,6 +1064,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach
var unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks)
verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true)
assert(memoryStore.currentUnrollMemoryForThisThread === 0)
+ memoryStore.releasePendingUnrollMemoryForThisThread()
// Unroll with not enough space. This should succeed after kicking out someBlock1.
store.putIterator("someBlock1", smallList.iterator, StorageLevel.MEMORY_ONLY)
@@ -1074,6 +1075,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach
assert(droppedBlocks.size === 1)
assert(droppedBlocks.head._1 === TestBlockId("someBlock1"))
droppedBlocks.clear()
+ memoryStore.releasePendingUnrollMemoryForThisThread()
// Unroll huge block with not enough space. Even after ensuring free space of 12000 * 0.4 =
// 4800 bytes, there is still not enough room to unroll this block. This returns an iterator.
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala
index bbc7e1357b90d..c21c92b63ad13 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala
@@ -31,6 +31,8 @@ class BlockObjectWriterSuite extends FunSuite {
new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics)
writer.write(Long.box(20))
+ // Record metrics update on every write
+ assert(writeMetrics.shuffleRecordsWritten === 1)
// Metrics don't update on every write
assert(writeMetrics.shuffleBytesWritten == 0)
// After 32 writes, metrics should update
@@ -39,6 +41,7 @@ class BlockObjectWriterSuite extends FunSuite {
writer.write(Long.box(i))
}
assert(writeMetrics.shuffleBytesWritten > 0)
+ assert(writeMetrics.shuffleRecordsWritten === 33)
writer.commitAndClose()
assert(file.length() == writeMetrics.shuffleBytesWritten)
}
@@ -51,6 +54,8 @@ class BlockObjectWriterSuite extends FunSuite {
new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics)
writer.write(Long.box(20))
+ // Record metrics update on every write
+ assert(writeMetrics.shuffleRecordsWritten === 1)
// Metrics don't update on every write
assert(writeMetrics.shuffleBytesWritten == 0)
// After 32 writes, metrics should update
@@ -59,7 +64,23 @@ class BlockObjectWriterSuite extends FunSuite {
writer.write(Long.box(i))
}
assert(writeMetrics.shuffleBytesWritten > 0)
+ assert(writeMetrics.shuffleRecordsWritten === 33)
writer.revertPartialWritesAndClose()
assert(writeMetrics.shuffleBytesWritten == 0)
+ assert(writeMetrics.shuffleRecordsWritten == 0)
+ }
+
+ test("Reopening a closed block writer") {
+ val file = new File("somefile")
+ file.deleteOnExit()
+ val writeMetrics = new ShuffleWriteMetrics()
+ val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file,
+ new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics)
+
+ writer.open()
+ writer.close()
+ intercept[IllegalStateException] {
+ writer.open()
+ }
}
}
diff --git a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala
index 8cf951adb354b..82a82e23eecf2 100644
--- a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.storage
import java.io.File
import org.apache.spark.util.Utils
-import org.scalatest.FunSuite
+import org.scalatest.{BeforeAndAfter, FunSuite}
import org.apache.spark.SparkConf
@@ -28,7 +28,11 @@ import org.apache.spark.SparkConf
/**
* Tests for the spark.local.dir and SPARK_LOCAL_DIRS configuration options.
*/
-class LocalDirsSuite extends FunSuite {
+class LocalDirsSuite extends FunSuite with BeforeAndAfter {
+
+ before {
+ Utils.clearLocalRootDirs()
+ }
test("Utils.getLocalDir() returns a valid directory, even if some local dirs are missing") {
// Regression test for SPARK-2974
diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
index e85a436cdba17..6a972381faf14 100644
--- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
@@ -32,12 +32,21 @@ import org.apache.spark.api.java.StorageLevels
import org.apache.spark.shuffle.FetchFailedException
/**
- * Selenium tests for the Spark Web UI. These tests are not run by default
- * because they're slow.
+ * Selenium tests for the Spark Web UI.
*/
-@DoNotDiscover
-class UISeleniumSuite extends FunSuite with WebBrowser with Matchers {
- implicit val webDriver: WebDriver = new HtmlUnitDriver
+class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll {
+
+ implicit var webDriver: WebDriver = _
+
+ override def beforeAll(): Unit = {
+ webDriver = new HtmlUnitDriver
+ }
+
+ override def afterAll(): Unit = {
+ if (webDriver != null) {
+ webDriver.quit()
+ }
+ }
/**
* Create a test SparkContext with the SparkUI enabled.
@@ -48,6 +57,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers {
.setMaster("local")
.setAppName("test")
.set("spark.ui.enabled", "true")
+ .set("spark.ui.port", "0")
val sc = new SparkContext(conf)
assert(sc.ui.isDefined)
sc
@@ -93,7 +103,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers {
}
eventually(timeout(5 seconds), interval(50 milliseconds)) {
go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages")
- find(id("active")).get.text should be("Active Stages (0)")
+ find(id("active")) should be(None) // Since we hide empty tables
find(id("failed")).get.text should be("Failed Stages (1)")
}
@@ -105,7 +115,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers {
}
eventually(timeout(5 seconds), interval(50 milliseconds)) {
go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages")
- find(id("active")).get.text should be("Active Stages (0)")
+ find(id("active")) should be(None) // Since we hide empty tables
// The failure occurs before the stage becomes active, hence we should still show only one
// failed stage, not two:
find(id("failed")).get.text should be("Failed Stages (1)")
@@ -167,13 +177,14 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers {
test("job progress bars should handle stage / task failures") {
withSpark(newSparkContext()) { sc =>
- val data = sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity)
+ val data = sc.parallelize(Seq(1, 2, 3), 1).map(identity).groupBy(identity)
val shuffleHandle =
data.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle
// Simulate fetch failures:
val mappedData = data.map { x =>
val taskContext = TaskContext.get
- if (taskContext.attemptNumber == 0) { // Cause this stage to fail on its first attempt.
+ if (taskContext.taskAttemptId() == 1) {
+ // Cause the post-shuffle stage to fail on its first attempt with a single task failure
val env = SparkEnv.get
val bmAddress = env.blockManager.blockManagerId
val shuffleId = shuffleHandle.shuffleId
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 68074ae32a672..730a4b54f5aa1 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
@@ -88,6 +88,28 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc
listener.completedStages.map(_.stageId).toSet should be (Set(50, 49, 48, 47, 46))
}
+ test("test clearing of stageIdToActiveJobs") {
+ val conf = new SparkConf()
+ conf.set("spark.ui.retainedStages", 5.toString)
+ val listener = new JobProgressListener(conf)
+ val jobId = 0
+ val stageIds = 1 to 50
+ // Start a job with 50 stages
+ listener.onJobStart(createJobStartEvent(jobId, stageIds))
+ for (stageId <- stageIds) {
+ listener.onStageSubmitted(createStageStartEvent(stageId))
+ }
+ listener.stageIdToActiveJobIds.size should be > 0
+
+ // Complete the stages and job
+ for (stageId <- stageIds) {
+ listener.onStageCompleted(createStageEndEvent(stageId, failed = false))
+ }
+ listener.onJobEnd(createJobEndEvent(jobId, false))
+ assertActiveJobsStateIsEmpty(listener)
+ listener.stageIdToActiveJobIds.size should be (0)
+ }
+
test("test LRU eviction of jobs") {
val conf = new SparkConf()
conf.set("spark.ui.retainedStages", 5.toString)
@@ -227,6 +249,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc
taskMetrics.setShuffleReadMetrics(Some(shuffleReadMetrics))
taskMetrics.shuffleWriteMetrics = Some(shuffleWriteMetrics)
shuffleReadMetrics.incRemoteBytesRead(base + 1)
+ shuffleReadMetrics.incLocalBytesRead(base + 9)
shuffleReadMetrics.incRemoteBlocksFetched(base + 2)
shuffleWriteMetrics.incShuffleBytesWritten(base + 3)
taskMetrics.setExecutorRunTime(base + 4)
@@ -234,7 +257,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc
taskMetrics.incMemoryBytesSpilled(base + 6)
val inputMetrics = new InputMetrics(DataReadMethod.Hadoop)
taskMetrics.setInputMetrics(Some(inputMetrics))
- inputMetrics.addBytesRead(base + 7)
+ inputMetrics.incBytesRead(base + 7)
val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop)
taskMetrics.outputMetrics = Some(outputMetrics)
outputMetrics.setBytesWritten(base + 8)
@@ -260,8 +283,8 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc
var stage0Data = listener.stageIdToData.get((0, 0)).get
var stage1Data = listener.stageIdToData.get((1, 0)).get
- assert(stage0Data.shuffleReadBytes == 102)
- assert(stage1Data.shuffleReadBytes == 201)
+ assert(stage0Data.shuffleReadTotalBytes == 220)
+ assert(stage1Data.shuffleReadTotalBytes == 410)
assert(stage0Data.shuffleWriteBytes == 106)
assert(stage1Data.shuffleWriteBytes == 203)
assert(stage0Data.executorRunTime == 108)
@@ -290,8 +313,11 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc
stage0Data = listener.stageIdToData.get((0, 0)).get
stage1Data = listener.stageIdToData.get((1, 0)).get
- assert(stage0Data.shuffleReadBytes == 402)
- assert(stage1Data.shuffleReadBytes == 602)
+ // Task 1235 contributed (100+1)+(100+9) = 210 shuffle bytes, and task 1234 contributed
+ // (300+1)+(300+9) = 610 total shuffle bytes, so the total for the stage is 820.
+ assert(stage0Data.shuffleReadTotalBytes == 820)
+ // Task 1236 contributed 410 shuffle bytes, and task 1237 contributed 810 shuffle bytes.
+ assert(stage1Data.shuffleReadTotalBytes == 1220)
assert(stage0Data.shuffleWriteBytes == 406)
assert(stage1Data.shuffleWriteBytes == 606)
assert(stage0Data.executorRunTime == 408)
diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala
index 6bbf72e929dcb..6250d50fb7036 100644
--- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala
@@ -17,7 +17,10 @@
package org.apache.spark.util
+import java.util.concurrent.TimeoutException
+
import scala.concurrent.Await
+import scala.util.{Failure, Try}
import akka.actor._
@@ -26,6 +29,7 @@ import org.scalatest.FunSuite
import org.apache.spark._
import org.apache.spark.scheduler.MapStatus
import org.apache.spark.storage.BlockManagerId
+import org.apache.spark.SSLSampleConfigs._
/**
@@ -47,7 +51,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro
val masterTracker = new MapOutputTrackerMaster(conf)
masterTracker.trackerActor = actorSystem.actorOf(
- Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker")
+ Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker")
val badconf = new SparkConf
badconf.set("spark.authenticate", "true")
@@ -60,7 +64,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro
conf = conf, securityManager = securityManagerBad)
val slaveTracker = new MapOutputTrackerWorker(conf)
val selection = slaveSystem.actorSelection(
- s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker")
+ AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker"))
val timeout = AkkaUtils.lookupTimeout(conf)
intercept[akka.actor.ActorNotFound] {
slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout)
@@ -74,7 +78,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro
val conf = new SparkConf
conf.set("spark.authenticate", "false")
conf.set("spark.authenticate.secret", "bad")
- val securityManager = new SecurityManager(conf);
+ val securityManager = new SecurityManager(conf)
val hostname = "localhost"
val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0,
@@ -85,18 +89,18 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro
val masterTracker = new MapOutputTrackerMaster(conf)
masterTracker.trackerActor = actorSystem.actorOf(
- Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker")
+ Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker")
val badconf = new SparkConf
badconf.set("spark.authenticate", "false")
badconf.set("spark.authenticate.secret", "good")
- val securityManagerBad = new SecurityManager(badconf);
+ val securityManagerBad = new SecurityManager(badconf)
val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0,
conf = badconf, securityManager = securityManagerBad)
val slaveTracker = new MapOutputTrackerWorker(conf)
val selection = slaveSystem.actorSelection(
- s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker")
+ AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker"))
val timeout = AkkaUtils.lookupTimeout(conf)
slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout)
@@ -124,7 +128,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro
val conf = new SparkConf
conf.set("spark.authenticate", "true")
conf.set("spark.authenticate.secret", "good")
- val securityManager = new SecurityManager(conf);
+ val securityManager = new SecurityManager(conf)
val hostname = "localhost"
val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0,
@@ -135,12 +139,12 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro
val masterTracker = new MapOutputTrackerMaster(conf)
masterTracker.trackerActor = actorSystem.actorOf(
- Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker")
+ Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker")
val goodconf = new SparkConf
goodconf.set("spark.authenticate", "true")
goodconf.set("spark.authenticate.secret", "good")
- val securityManagerGood = new SecurityManager(goodconf);
+ val securityManagerGood = new SecurityManager(goodconf)
assert(securityManagerGood.isAuthenticationEnabled() === true)
@@ -148,7 +152,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro
conf = goodconf, securityManager = securityManagerGood)
val slaveTracker = new MapOutputTrackerWorker(conf)
val selection = slaveSystem.actorSelection(
- s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker")
+ AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker"))
val timeout = AkkaUtils.lookupTimeout(conf)
slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout)
@@ -175,7 +179,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro
conf.set("spark.authenticate", "true")
conf.set("spark.authenticate.secret", "good")
- val securityManager = new SecurityManager(conf);
+ val securityManager = new SecurityManager(conf)
val hostname = "localhost"
val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0,
@@ -186,12 +190,12 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro
val masterTracker = new MapOutputTrackerMaster(conf)
masterTracker.trackerActor = actorSystem.actorOf(
- Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker")
+ Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker")
val badconf = new SparkConf
badconf.set("spark.authenticate", "false")
badconf.set("spark.authenticate.secret", "bad")
- val securityManagerBad = new SecurityManager(badconf);
+ val securityManagerBad = new SecurityManager(badconf)
assert(securityManagerBad.isAuthenticationEnabled() === false)
@@ -199,7 +203,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro
conf = badconf, securityManager = securityManagerBad)
val slaveTracker = new MapOutputTrackerWorker(conf)
val selection = slaveSystem.actorSelection(
- s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker")
+ AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker"))
val timeout = AkkaUtils.lookupTimeout(conf)
intercept[akka.actor.ActorNotFound] {
slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout)
@@ -209,4 +213,174 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro
slaveSystem.shutdown()
}
+ test("remote fetch ssl on") {
+ val conf = sparkSSLConfig()
+ val securityManager = new SecurityManager(conf)
+
+ val hostname = "localhost"
+ val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0,
+ conf = conf, securityManager = securityManager)
+ System.setProperty("spark.hostPort", hostname + ":" + boundPort)
+
+ assert(securityManager.isAuthenticationEnabled() === false)
+
+ val masterTracker = new MapOutputTrackerMaster(conf)
+ masterTracker.trackerActor = actorSystem.actorOf(
+ Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker")
+
+ val slaveConf = sparkSSLConfig()
+ val securityManagerBad = new SecurityManager(slaveConf)
+
+ val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0,
+ conf = slaveConf, securityManager = securityManagerBad)
+ val slaveTracker = new MapOutputTrackerWorker(conf)
+ val selection = slaveSystem.actorSelection(
+ AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker"))
+ val timeout = AkkaUtils.lookupTimeout(conf)
+ slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout)
+
+ assert(securityManagerBad.isAuthenticationEnabled() === false)
+
+ masterTracker.registerShuffle(10, 1)
+ masterTracker.incrementEpoch()
+ slaveTracker.updateEpoch(masterTracker.getEpoch)
+
+ val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L))
+ masterTracker.registerMapOutput(10, 0,
+ MapStatus(BlockManagerId("a", "hostA", 1000), Array(1000L)))
+ masterTracker.incrementEpoch()
+ slaveTracker.updateEpoch(masterTracker.getEpoch)
+
+ // this should succeed since security off
+ assert(slaveTracker.getServerStatuses(10, 0).toSeq ===
+ Seq((BlockManagerId("a", "hostA", 1000), size1000)))
+
+ actorSystem.shutdown()
+ slaveSystem.shutdown()
+ }
+
+
+ test("remote fetch ssl on and security enabled") {
+ val conf = sparkSSLConfig()
+ conf.set("spark.authenticate", "true")
+ conf.set("spark.authenticate.secret", "good")
+ val securityManager = new SecurityManager(conf)
+
+ val hostname = "localhost"
+ val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0,
+ conf = conf, securityManager = securityManager)
+ System.setProperty("spark.hostPort", hostname + ":" + boundPort)
+
+ assert(securityManager.isAuthenticationEnabled() === true)
+
+ val masterTracker = new MapOutputTrackerMaster(conf)
+ masterTracker.trackerActor = actorSystem.actorOf(
+ Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker")
+
+ val slaveConf = sparkSSLConfig()
+ slaveConf.set("spark.authenticate", "true")
+ slaveConf.set("spark.authenticate.secret", "good")
+ val securityManagerBad = new SecurityManager(slaveConf)
+
+ val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0,
+ conf = slaveConf, securityManager = securityManagerBad)
+ val slaveTracker = new MapOutputTrackerWorker(conf)
+ val selection = slaveSystem.actorSelection(
+ AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker"))
+ val timeout = AkkaUtils.lookupTimeout(conf)
+ slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout)
+
+ assert(securityManagerBad.isAuthenticationEnabled() === true)
+
+ masterTracker.registerShuffle(10, 1)
+ masterTracker.incrementEpoch()
+ slaveTracker.updateEpoch(masterTracker.getEpoch)
+
+ val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L))
+ masterTracker.registerMapOutput(10, 0,
+ MapStatus(BlockManagerId("a", "hostA", 1000), Array(1000L)))
+ masterTracker.incrementEpoch()
+ slaveTracker.updateEpoch(masterTracker.getEpoch)
+
+ assert(slaveTracker.getServerStatuses(10, 0).toSeq ===
+ Seq((BlockManagerId("a", "hostA", 1000), size1000)))
+
+ actorSystem.shutdown()
+ slaveSystem.shutdown()
+ }
+
+
+ test("remote fetch ssl on and security enabled - bad credentials") {
+ val conf = sparkSSLConfig()
+ conf.set("spark.authenticate", "true")
+ conf.set("spark.authenticate.secret", "good")
+ val securityManager = new SecurityManager(conf)
+
+ val hostname = "localhost"
+ val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0,
+ conf = conf, securityManager = securityManager)
+ System.setProperty("spark.hostPort", hostname + ":" + boundPort)
+
+ assert(securityManager.isAuthenticationEnabled() === true)
+
+ val masterTracker = new MapOutputTrackerMaster(conf)
+ masterTracker.trackerActor = actorSystem.actorOf(
+ Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker")
+
+ val slaveConf = sparkSSLConfig()
+ slaveConf.set("spark.authenticate", "true")
+ slaveConf.set("spark.authenticate.secret", "bad")
+ val securityManagerBad = new SecurityManager(slaveConf)
+
+ val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0,
+ conf = slaveConf, securityManager = securityManagerBad)
+ val slaveTracker = new MapOutputTrackerWorker(conf)
+ val selection = slaveSystem.actorSelection(
+ AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker"))
+ val timeout = AkkaUtils.lookupTimeout(conf)
+ intercept[akka.actor.ActorNotFound] {
+ slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout)
+ }
+
+ actorSystem.shutdown()
+ slaveSystem.shutdown()
+ }
+
+
+ test("remote fetch ssl on - untrusted server") {
+ val conf = sparkSSLConfigUntrusted()
+ val securityManager = new SecurityManager(conf)
+
+ val hostname = "localhost"
+ val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0,
+ conf = conf, securityManager = securityManager)
+ System.setProperty("spark.hostPort", hostname + ":" + boundPort)
+
+ assert(securityManager.isAuthenticationEnabled() === false)
+
+ val masterTracker = new MapOutputTrackerMaster(conf)
+ masterTracker.trackerActor = actorSystem.actorOf(
+ Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker")
+
+ val slaveConf = sparkSSLConfig()
+ val securityManagerBad = new SecurityManager(slaveConf)
+
+ val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0,
+ conf = slaveConf, securityManager = securityManagerBad)
+ val slaveTracker = new MapOutputTrackerWorker(conf)
+ val selection = slaveSystem.actorSelection(
+ AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker"))
+ val timeout = AkkaUtils.lookupTimeout(conf)
+ val result = Try(Await.result(selection.resolveOne(timeout * 2), timeout))
+
+ result match {
+ case Failure(ex: ActorNotFound) =>
+ case Failure(ex: TimeoutException) =>
+ case r => fail(s"$r is neither Failure(ActorNotFound) nor Failure(TimeoutException)")
+ }
+
+ actorSystem.shutdown()
+ slaveSystem.shutdown()
+ }
+
}
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 6577ebaa2e9a8..a2be724254d7c 100644
--- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
@@ -76,8 +76,9 @@ class JsonProtocolSuite extends FunSuite {
val unpersistRdd = SparkListenerUnpersistRDD(12345)
val applicationStart = SparkListenerApplicationStart("The winner of all", None, 42L, "Garfield")
val applicationEnd = SparkListenerApplicationEnd(42L)
+ val logUrlMap = Map("stderr" -> "mystderr", "stdout" -> "mystdout").toMap
val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1",
- new ExecutorInfo("Hostee.awesome.com", 11))
+ new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap))
val executorRemoved = SparkListenerExecutorRemoved(executorRemovedTime, "exec2", "test reason")
testEvent(stageSubmitted, stageSubmittedJsonString)
@@ -100,13 +101,14 @@ class JsonProtocolSuite extends FunSuite {
}
test("Dependent Classes") {
+ val logUrlMap = Map("stderr" -> "mystderr", "stdout" -> "mystdout").toMap
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, hasHadoopInput = false, hasOutput = false))
testBlockManagerId(BlockManagerId("Hong", "Kong", 500))
- testExecutorInfo(new ExecutorInfo("host", 43))
+ testExecutorInfo(new ExecutorInfo("host", 43, logUrlMap))
// StorageLevel
testStorageLevel(StorageLevel.NONE)
@@ -187,6 +189,34 @@ class JsonProtocolSuite extends FunSuite {
assert(newMetrics.inputMetrics.isEmpty)
}
+ test("Input/Output records backwards compatibility") {
+ // records read were added after 1.2
+ val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6,
+ hasHadoopInput = true, hasOutput = true, hasRecords = false)
+ assert(metrics.inputMetrics.nonEmpty)
+ assert(metrics.outputMetrics.nonEmpty)
+ val newJson = JsonProtocol.taskMetricsToJson(metrics)
+ val oldJson = newJson.removeField { case (field, _) => field == "Records Read" }
+ .removeField { case (field, _) => field == "Records Written" }
+ val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson)
+ assert(newMetrics.inputMetrics.get.recordsRead == 0)
+ assert(newMetrics.outputMetrics.get.recordsWritten == 0)
+ }
+
+ test("Shuffle Read/Write records backwards compatibility") {
+ // records read were added after 1.2
+ val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6,
+ hasHadoopInput = false, hasOutput = false, hasRecords = false)
+ assert(metrics.shuffleReadMetrics.nonEmpty)
+ assert(metrics.shuffleWriteMetrics.nonEmpty)
+ val newJson = JsonProtocol.taskMetricsToJson(metrics)
+ val oldJson = newJson.removeField { case (field, _) => field == "Total Records Read" }
+ .removeField { case (field, _) => field == "Shuffle Records Written" }
+ val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson)
+ assert(newMetrics.shuffleReadMetrics.get.recordsRead == 0)
+ assert(newMetrics.shuffleWriteMetrics.get.shuffleRecordsWritten == 0)
+ }
+
test("OutputMetrics backward compatibility") {
// OutputMetrics were added after 1.1
val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6, hasHadoopInput = false, hasOutput = true)
@@ -230,6 +260,18 @@ class JsonProtocolSuite extends FunSuite {
assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent))
}
+ test("ShuffleReadMetrics: Local bytes read and time taken backwards compatibility") {
+ // Metrics about local shuffle bytes read and local read time were added in 1.3.1.
+ val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6,
+ hasHadoopInput = false, hasOutput = false, hasRecords = false)
+ assert(metrics.shuffleReadMetrics.nonEmpty)
+ val newJson = JsonProtocol.taskMetricsToJson(metrics)
+ val oldJson = newJson.removeField { case (field, _) => field == "Local Bytes Read" }
+ .removeField { case (field, _) => field == "Local Read Time" }
+ val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson)
+ assert(newMetrics.shuffleReadMetrics.get.localBytesRead == 0)
+ }
+
test("SparkListenerApplicationStart backwards compatibility") {
// SparkListenerApplicationStart in Spark 1.0.0 do not have an "appId" property.
val applicationStart = SparkListenerApplicationStart("test", None, 1L, "user")
@@ -642,7 +684,8 @@ class JsonProtocolSuite extends FunSuite {
e: Int,
f: Int,
hasHadoopInput: Boolean,
- hasOutput: Boolean) = {
+ hasOutput: Boolean,
+ hasRecords: Boolean = true) = {
val t = new TaskMetrics
t.setHostname("localhost")
t.setExecutorDeserializeTime(a)
@@ -654,7 +697,8 @@ class JsonProtocolSuite extends FunSuite {
if (hasHadoopInput) {
val inputMetrics = new InputMetrics(DataReadMethod.Hadoop)
- inputMetrics.addBytesRead(d + e + f)
+ inputMetrics.incBytesRead(d + e + f)
+ inputMetrics.incRecordsRead(if (hasRecords) (d + e + f) / 100 else -1)
t.setInputMetrics(Some(inputMetrics))
} else {
val sr = new ShuffleReadMetrics
@@ -662,16 +706,20 @@ class JsonProtocolSuite extends FunSuite {
sr.incLocalBlocksFetched(e)
sr.incFetchWaitTime(a + d)
sr.incRemoteBlocksFetched(f)
+ sr.incRecordsRead(if (hasRecords) (b + d) / 100 else -1)
+ sr.incLocalBytesRead(a + f)
t.setShuffleReadMetrics(Some(sr))
}
if (hasOutput) {
val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop)
outputMetrics.setBytesWritten(a + b + c)
+ outputMetrics.setRecordsWritten(if (hasRecords) (a + b + c)/100 else -1)
t.outputMetrics = Some(outputMetrics)
} else {
val sw = new ShuffleWriteMetrics
sw.incShuffleBytesWritten(a + b + c)
sw.incShuffleWriteTime(b + c + d)
+ sw.setShuffleRecordsWritten(if (hasRecords) (a + b + c) / 100 else -1)
t.shuffleWriteMetrics = Some(sw)
}
// Make at most 6 blocks
@@ -905,11 +953,14 @@ class JsonProtocolSuite extends FunSuite {
| "Remote Blocks Fetched": 800,
| "Local Blocks Fetched": 700,
| "Fetch Wait Time": 900,
- | "Remote Bytes Read": 1000
+ | "Remote Bytes Read": 1000,
+ | "Local Bytes Read": 1100,
+ | "Total Records Read" : 10
| },
| "Shuffle Write Metrics": {
| "Shuffle Bytes Written": 1200,
- | "Shuffle Write Time": 1500
+ | "Shuffle Write Time": 1500,
+ | "Shuffle Records Written": 12
| },
| "Updated Blocks": [
| {
@@ -986,11 +1037,13 @@ class JsonProtocolSuite extends FunSuite {
| "Disk Bytes Spilled": 0,
| "Shuffle Write Metrics": {
| "Shuffle Bytes Written": 1200,
- | "Shuffle Write Time": 1500
+ | "Shuffle Write Time": 1500,
+ | "Shuffle Records Written": 12
| },
| "Input Metrics": {
| "Data Read Method": "Hadoop",
- | "Bytes Read": 2100
+ | "Bytes Read": 2100,
+ | "Records Read": 21
| },
| "Updated Blocks": [
| {
@@ -1067,11 +1120,13 @@ class JsonProtocolSuite extends FunSuite {
| "Disk Bytes Spilled": 0,
| "Input Metrics": {
| "Data Read Method": "Hadoop",
- | "Bytes Read": 2100
+ | "Bytes Read": 2100,
+ | "Records Read": 21
| },
| "Output Metrics": {
| "Data Write Method": "Hadoop",
- | "Bytes Written": 1200
+ | "Bytes Written": 1200,
+ | "Records Written": 12
| },
| "Updated Blocks": [
| {
@@ -1463,7 +1518,11 @@ class JsonProtocolSuite extends FunSuite {
| "Executor ID": "exec1",
| "Executor Info": {
| "Host": "Hostee.awesome.com",
- | "Total Cores": 11
+ | "Total Cores": 11,
+ | "Log Urls" : {
+ | "stderr" : "mystderr",
+ | "stdout" : "mystdout"
+ | }
| }
|}
"""
diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala
similarity index 73%
rename from core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala
rename to core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala
index e2050e95a1b88..87de90bb0dfb0 100644
--- a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala
@@ -15,41 +15,48 @@
* limitations under the License.
*/
-package org.apache.spark.executor
+package org.apache.spark.util
import java.net.URLClassLoader
import org.scalatest.FunSuite
-import org.apache.spark.{LocalSparkContext, SparkContext, SparkException, TestUtils}
-import org.apache.spark.util.Utils
+import org.apache.spark.{SparkContext, SparkException, TestUtils}
-class ExecutorURLClassLoaderSuite extends FunSuite {
+class MutableURLClassLoaderSuite extends FunSuite {
- val childClassNames = List("FakeClass1", "FakeClass2")
- val parentClassNames = List("FakeClass1", "FakeClass2", "FakeClass3")
- val urls = List(TestUtils.createJarWithClasses(childClassNames, "1")).toArray
- val urls2 = List(TestUtils.createJarWithClasses(parentClassNames, "2")).toArray
+ val urls2 = List(TestUtils.createJarWithClasses(
+ classNames = Seq("FakeClass1", "FakeClass2", "FakeClass3"),
+ toStringValue = "2")).toArray
+ val urls = List(TestUtils.createJarWithClasses(
+ classNames = Seq("FakeClass1"),
+ classNamesWithBase = Seq(("FakeClass2", "FakeClass3")), // FakeClass3 is in parent
+ toStringValue = "1",
+ classpathUrls = urls2)).toArray
test("child first") {
val parentLoader = new URLClassLoader(urls2, null)
- val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader)
+ val classLoader = new ChildFirstURLClassLoader(urls, parentLoader)
val fakeClass = classLoader.loadClass("FakeClass2").newInstance()
val fakeClassVersion = fakeClass.toString
assert(fakeClassVersion === "1")
+ val fakeClass2 = classLoader.loadClass("FakeClass2").newInstance()
+ assert(fakeClass.getClass === fakeClass2.getClass)
}
test("parent first") {
val parentLoader = new URLClassLoader(urls2, null)
- val classLoader = new ExecutorURLClassLoader(urls, parentLoader)
+ val classLoader = new MutableURLClassLoader(urls, parentLoader)
val fakeClass = classLoader.loadClass("FakeClass1").newInstance()
val fakeClassVersion = fakeClass.toString
assert(fakeClassVersion === "2")
+ val fakeClass2 = classLoader.loadClass("FakeClass1").newInstance()
+ assert(fakeClass.getClass === fakeClass2.getClass)
}
test("child first can fall back") {
val parentLoader = new URLClassLoader(urls2, null)
- val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader)
+ val classLoader = new ChildFirstURLClassLoader(urls, parentLoader)
val fakeClass = classLoader.loadClass("FakeClass3").newInstance()
val fakeClassVersion = fakeClass.toString
assert(fakeClassVersion === "2")
@@ -57,7 +64,7 @@ class ExecutorURLClassLoaderSuite extends FunSuite {
test("child first can fail") {
val parentLoader = new URLClassLoader(urls2, null)
- val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader)
+ val classLoader = new ChildFirstURLClassLoader(urls, parentLoader)
intercept[java.lang.ClassNotFoundException] {
classLoader.loadClass("FakeClassDoesNotExist").newInstance()
}
diff --git a/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala
index d4b92f33dd9e6..bad1aa99952cf 100644
--- a/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala
+++ b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala
@@ -19,6 +19,7 @@ package org.apache.spark.util
import java.util.Properties
+import org.apache.commons.lang3.SerializationUtils
import org.scalatest.{BeforeAndAfterEach, Suite}
/**
@@ -42,7 +43,11 @@ private[spark] trait ResetSystemProperties extends BeforeAndAfterEach { this: Su
var oldProperties: Properties = null
override def beforeEach(): Unit = {
- oldProperties = new Properties(System.getProperties)
+ // we need SerializationUtils.clone instead of `new Properties(System.getProperties()` because
+ // the later way of creating a copy does not copy the properties but it initializes a new
+ // Properties object with the given properties as defaults. They are not recognized at all
+ // by standard Scala wrapper over Java Properties then.
+ oldProperties = SerializationUtils.clone(System.getProperties)
super.beforeEach()
}
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index 4544382094f96..b91428efadfd0 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -29,6 +29,9 @@ import com.google.common.base.Charsets.UTF_8
import com.google.common.io.Files
import org.scalatest.FunSuite
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
import org.apache.spark.SparkConf
class UtilsSuite extends FunSuite with ResetSystemProperties {
@@ -205,18 +208,18 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
child1.setLastModified(System.currentTimeMillis() - (1000 * 30))
// although child1 is old, child2 is still new so return true
- assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5))
+ assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5))
child2.setLastModified(System.currentTimeMillis - (1000 * 30))
- assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5))
+ assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5))
parent.setLastModified(System.currentTimeMillis - (1000 * 30))
// although parent and its immediate children are new, child3 is still old
// we expect a full recursive search for new files.
- assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5))
+ assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5))
child3.setLastModified(System.currentTimeMillis - (1000 * 30))
- assert(!Utils.doesDirectoryContainAnyNewFiles(parent, 5))
+ assert(!Utils.doesDirectoryContainAnyNewFiles(parent, 5))
}
test("resolveURI") {
@@ -336,21 +339,21 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
assert(!tempDir1.exists())
val tempDir2 = Utils.createTempDir()
- val tempFile1 = new File(tempDir2, "foo.txt")
- Files.touch(tempFile1)
- assert(tempFile1.exists())
- Utils.deleteRecursively(tempFile1)
- assert(!tempFile1.exists())
+ val sourceFile1 = new File(tempDir2, "foo.txt")
+ Files.touch(sourceFile1)
+ assert(sourceFile1.exists())
+ Utils.deleteRecursively(sourceFile1)
+ assert(!sourceFile1.exists())
val tempDir3 = new File(tempDir2, "subdir")
assert(tempDir3.mkdir())
- val tempFile2 = new File(tempDir3, "bar.txt")
- Files.touch(tempFile2)
- assert(tempFile2.exists())
+ val sourceFile2 = new File(tempDir3, "bar.txt")
+ Files.touch(sourceFile2)
+ assert(sourceFile2.exists())
Utils.deleteRecursively(tempDir2)
assert(!tempDir2.exists())
assert(!tempDir3.exists())
- assert(!tempFile2.exists())
+ assert(!sourceFile2.exists())
}
test("loading properties from file") {
@@ -381,4 +384,42 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
require(cnt === 2, "prepare should be called twice")
require(time < 500, "preparation time should not count")
}
+
+ test("fetch hcfs dir") {
+ val tempDir = Utils.createTempDir()
+ val sourceDir = new File(tempDir, "source-dir")
+ val innerSourceDir = Utils.createTempDir(root=sourceDir.getPath)
+ val sourceFile = File.createTempFile("someprefix", "somesuffix", innerSourceDir)
+ val targetDir = new File(tempDir, "target-dir")
+ Files.write("some text", sourceFile, UTF_8)
+
+ val path = new Path("file://" + sourceDir.getAbsolutePath)
+ val conf = new Configuration()
+ val fs = Utils.getHadoopFileSystem(path.toString, conf)
+
+ assert(!targetDir.isDirectory())
+ Utils.fetchHcfsFile(path, targetDir, fs, new SparkConf(), conf, false)
+ assert(targetDir.isDirectory())
+
+ // Copy again to make sure it doesn't error if the dir already exists.
+ Utils.fetchHcfsFile(path, targetDir, fs, new SparkConf(), conf, false)
+
+ val destDir = new File(targetDir, sourceDir.getName())
+ assert(destDir.isDirectory())
+
+ val destInnerDir = new File(destDir, innerSourceDir.getName)
+ assert(destInnerDir.isDirectory())
+
+ val destInnerFile = new File(destInnerDir, sourceFile.getName)
+ assert(destInnerFile.isFile())
+
+ val filePath = new Path("file://" + sourceFile.getAbsolutePath)
+ val testFileDir = new File(tempDir, "test-filename")
+ val testFileName = "testFName"
+ val testFilefs = Utils.getHadoopFileSystem(filePath.toString, conf)
+ Utils.fetchHcfsFile(filePath, testFileDir, testFilefs, new SparkConf(),
+ conf, false, Some(testFileName))
+ val newFileName = new File(testFileDir, testFileName)
+ assert(newFileName.isFile())
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala
index 0cb1ed7397655..e0d6cc16bde05 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala
@@ -65,6 +65,13 @@ class SorterSuite extends FunSuite {
}
}
+ // http://www.envisage-project.eu/timsort-specification-and-verification/
+ test("SPARK-5984 TimSort bug") {
+ val data = TestTimSort.getTimSortBugTestSet(67108864)
+ new Sorter(new IntArraySortDataFormat).sort(data, 0, data.length, Ordering.Int)
+ (0 to data.length - 2).foreach(i => assert(data(i) <= data(i + 1)))
+ }
+
/** Runs an experiment several times. */
def runExperiment(name: String, skip: Boolean = false)(f: => Unit, prepare: () => Unit): Unit = {
if (skip) {
diff --git a/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala b/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala
index 4918e2d92beb4..daa795a043495 100644
--- a/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala
+++ b/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala
@@ -44,13 +44,21 @@ class ImplicitSuite {
}
def testRddToSequenceFileRDDFunctions(): Unit = {
- // TODO eliminating `import intToIntWritable` needs refactoring SequenceFileRDDFunctions.
- // That will be a breaking change.
- import org.apache.spark.SparkContext.intToIntWritable
val rdd: org.apache.spark.rdd.RDD[(Int, Int)] = mockRDD
rdd.saveAsSequenceFile("/a/test/path")
}
+ def testRddToSequenceFileRDDFunctionsWithWritable(): Unit = {
+ val rdd: org.apache.spark.rdd.RDD[(org.apache.hadoop.io.IntWritable, org.apache.hadoop.io.Text)]
+ = mockRDD
+ rdd.saveAsSequenceFile("/a/test/path")
+ }
+
+ def testRddToSequenceFileRDDFunctionsWithBytesArray(): Unit = {
+ val rdd: org.apache.spark.rdd.RDD[(Int, Array[Byte])] = mockRDD
+ rdd.saveAsSequenceFile("/a/test/path")
+ }
+
def testRddToOrderedRDDFunctions(): Unit = {
val rdd: org.apache.spark.rdd.RDD[(Int, Int)] = mockRDD
rdd.sortByKey()
diff --git a/data/mllib/als/sample_movielens_movies.txt b/data/mllib/als/sample_movielens_movies.txt
new file mode 100644
index 0000000000000..934a0253849e1
--- /dev/null
+++ b/data/mllib/als/sample_movielens_movies.txt
@@ -0,0 +1,100 @@
+0::Movie 0::Romance|Comedy
+1::Movie 1::Action|Anime
+2::Movie 2::Romance|Thriller
+3::Movie 3::Action|Romance
+4::Movie 4::Anime|Comedy
+5::Movie 5::Action|Action
+6::Movie 6::Action|Comedy
+7::Movie 7::Anime|Comedy
+8::Movie 8::Comedy|Action
+9::Movie 9::Anime|Thriller
+10::Movie 10::Action|Anime
+11::Movie 11::Action|Anime
+12::Movie 12::Anime|Comedy
+13::Movie 13::Thriller|Action
+14::Movie 14::Anime|Comedy
+15::Movie 15::Comedy|Thriller
+16::Movie 16::Anime|Romance
+17::Movie 17::Thriller|Action
+18::Movie 18::Action|Comedy
+19::Movie 19::Anime|Romance
+20::Movie 20::Action|Anime
+21::Movie 21::Romance|Thriller
+22::Movie 22::Romance|Romance
+23::Movie 23::Comedy|Comedy
+24::Movie 24::Anime|Action
+25::Movie 25::Comedy|Comedy
+26::Movie 26::Anime|Romance
+27::Movie 27::Anime|Anime
+28::Movie 28::Thriller|Anime
+29::Movie 29::Anime|Romance
+30::Movie 30::Thriller|Romance
+31::Movie 31::Thriller|Romance
+32::Movie 32::Comedy|Anime
+33::Movie 33::Comedy|Comedy
+34::Movie 34::Anime|Anime
+35::Movie 35::Action|Thriller
+36::Movie 36::Anime|Romance
+37::Movie 37::Romance|Anime
+38::Movie 38::Thriller|Romance
+39::Movie 39::Romance|Comedy
+40::Movie 40::Action|Anime
+41::Movie 41::Comedy|Thriller
+42::Movie 42::Comedy|Action
+43::Movie 43::Thriller|Anime
+44::Movie 44::Anime|Action
+45::Movie 45::Comedy|Romance
+46::Movie 46::Comedy|Action
+47::Movie 47::Romance|Comedy
+48::Movie 48::Action|Comedy
+49::Movie 49::Romance|Romance
+50::Movie 50::Comedy|Romance
+51::Movie 51::Action|Action
+52::Movie 52::Thriller|Action
+53::Movie 53::Action|Action
+54::Movie 54::Romance|Thriller
+55::Movie 55::Anime|Romance
+56::Movie 56::Comedy|Action
+57::Movie 57::Action|Anime
+58::Movie 58::Thriller|Romance
+59::Movie 59::Thriller|Comedy
+60::Movie 60::Anime|Comedy
+61::Movie 61::Comedy|Action
+62::Movie 62::Comedy|Romance
+63::Movie 63::Romance|Thriller
+64::Movie 64::Romance|Action
+65::Movie 65::Anime|Romance
+66::Movie 66::Comedy|Action
+67::Movie 67::Thriller|Anime
+68::Movie 68::Thriller|Romance
+69::Movie 69::Action|Comedy
+70::Movie 70::Thriller|Thriller
+71::Movie 71::Action|Comedy
+72::Movie 72::Thriller|Romance
+73::Movie 73::Comedy|Action
+74::Movie 74::Action|Action
+75::Movie 75::Action|Action
+76::Movie 76::Comedy|Comedy
+77::Movie 77::Comedy|Comedy
+78::Movie 78::Comedy|Comedy
+79::Movie 79::Thriller|Thriller
+80::Movie 80::Comedy|Anime
+81::Movie 81::Comedy|Anime
+82::Movie 82::Romance|Anime
+83::Movie 83::Comedy|Thriller
+84::Movie 84::Anime|Action
+85::Movie 85::Thriller|Anime
+86::Movie 86::Romance|Anime
+87::Movie 87::Thriller|Thriller
+88::Movie 88::Romance|Thriller
+89::Movie 89::Action|Anime
+90::Movie 90::Anime|Romance
+91::Movie 91::Anime|Thriller
+92::Movie 92::Action|Comedy
+93::Movie 93::Romance|Thriller
+94::Movie 94::Thriller|Comedy
+95::Movie 95::Action|Action
+96::Movie 96::Thriller|Romance
+97::Movie 97::Thriller|Thriller
+98::Movie 98::Thriller|Comedy
+99::Movie 99::Thriller|Romance
diff --git a/data/mllib/als/sample_movielens_ratings.txt b/data/mllib/als/sample_movielens_ratings.txt
new file mode 100644
index 0000000000000..0889142950797
--- /dev/null
+++ b/data/mllib/als/sample_movielens_ratings.txt
@@ -0,0 +1,1501 @@
+0::2::3::1424380312
+0::3::1::1424380312
+0::5::2::1424380312
+0::9::4::1424380312
+0::11::1::1424380312
+0::12::2::1424380312
+0::15::1::1424380312
+0::17::1::1424380312
+0::19::1::1424380312
+0::21::1::1424380312
+0::23::1::1424380312
+0::26::3::1424380312
+0::27::1::1424380312
+0::28::1::1424380312
+0::29::1::1424380312
+0::30::1::1424380312
+0::31::1::1424380312
+0::34::1::1424380312
+0::37::1::1424380312
+0::41::2::1424380312
+0::44::1::1424380312
+0::45::2::1424380312
+0::46::1::1424380312
+0::47::1::1424380312
+0::48::1::1424380312
+0::50::1::1424380312
+0::51::1::1424380312
+0::54::1::1424380312
+0::55::1::1424380312
+0::59::2::1424380312
+0::61::2::1424380312
+0::64::1::1424380312
+0::67::1::1424380312
+0::68::1::1424380312
+0::69::1::1424380312
+0::71::1::1424380312
+0::72::1::1424380312
+0::77::2::1424380312
+0::79::1::1424380312
+0::83::1::1424380312
+0::87::1::1424380312
+0::89::2::1424380312
+0::91::3::1424380312
+0::92::4::1424380312
+0::94::1::1424380312
+0::95::2::1424380312
+0::96::1::1424380312
+0::98::1::1424380312
+0::99::1::1424380312
+1::2::2::1424380312
+1::3::1::1424380312
+1::4::2::1424380312
+1::6::1::1424380312
+1::9::3::1424380312
+1::12::1::1424380312
+1::13::1::1424380312
+1::14::1::1424380312
+1::16::1::1424380312
+1::19::1::1424380312
+1::21::3::1424380312
+1::27::1::1424380312
+1::28::3::1424380312
+1::33::1::1424380312
+1::36::2::1424380312
+1::37::1::1424380312
+1::40::1::1424380312
+1::41::2::1424380312
+1::43::1::1424380312
+1::44::1::1424380312
+1::47::1::1424380312
+1::50::1::1424380312
+1::54::1::1424380312
+1::56::2::1424380312
+1::57::1::1424380312
+1::58::1::1424380312
+1::60::1::1424380312
+1::62::4::1424380312
+1::63::1::1424380312
+1::67::1::1424380312
+1::68::4::1424380312
+1::70::2::1424380312
+1::72::1::1424380312
+1::73::1::1424380312
+1::74::2::1424380312
+1::76::1::1424380312
+1::77::3::1424380312
+1::78::1::1424380312
+1::81::1::1424380312
+1::82::1::1424380312
+1::85::3::1424380312
+1::86::2::1424380312
+1::88::2::1424380312
+1::91::1::1424380312
+1::92::2::1424380312
+1::93::1::1424380312
+1::94::2::1424380312
+1::96::1::1424380312
+1::97::1::1424380312
+2::4::3::1424380312
+2::6::1::1424380312
+2::8::5::1424380312
+2::9::1::1424380312
+2::10::1::1424380312
+2::12::3::1424380312
+2::13::1::1424380312
+2::15::2::1424380312
+2::18::2::1424380312
+2::19::4::1424380312
+2::22::1::1424380312
+2::26::1::1424380312
+2::28::1::1424380312
+2::34::4::1424380312
+2::35::1::1424380312
+2::37::5::1424380312
+2::38::1::1424380312
+2::39::5::1424380312
+2::40::4::1424380312
+2::47::1::1424380312
+2::50::1::1424380312
+2::52::2::1424380312
+2::54::1::1424380312
+2::55::1::1424380312
+2::57::2::1424380312
+2::58::2::1424380312
+2::59::1::1424380312
+2::61::1::1424380312
+2::62::1::1424380312
+2::64::1::1424380312
+2::65::1::1424380312
+2::66::3::1424380312
+2::68::1::1424380312
+2::71::3::1424380312
+2::76::1::1424380312
+2::77::1::1424380312
+2::78::1::1424380312
+2::80::1::1424380312
+2::83::5::1424380312
+2::85::1::1424380312
+2::87::2::1424380312
+2::88::1::1424380312
+2::89::4::1424380312
+2::90::1::1424380312
+2::92::4::1424380312
+2::93::5::1424380312
+3::0::1::1424380312
+3::1::1::1424380312
+3::2::1::1424380312
+3::7::3::1424380312
+3::8::3::1424380312
+3::9::1::1424380312
+3::14::1::1424380312
+3::15::1::1424380312
+3::16::1::1424380312
+3::18::4::1424380312
+3::19::1::1424380312
+3::24::3::1424380312
+3::26::1::1424380312
+3::29::3::1424380312
+3::33::1::1424380312
+3::34::3::1424380312
+3::35::1::1424380312
+3::36::3::1424380312
+3::37::1::1424380312
+3::38::2::1424380312
+3::43::1::1424380312
+3::44::1::1424380312
+3::46::1::1424380312
+3::47::1::1424380312
+3::51::5::1424380312
+3::52::3::1424380312
+3::56::1::1424380312
+3::58::1::1424380312
+3::60::3::1424380312
+3::62::1::1424380312
+3::65::2::1424380312
+3::66::1::1424380312
+3::67::1::1424380312
+3::68::2::1424380312
+3::70::1::1424380312
+3::72::2::1424380312
+3::76::3::1424380312
+3::79::3::1424380312
+3::80::4::1424380312
+3::81::1::1424380312
+3::83::1::1424380312
+3::84::1::1424380312
+3::86::1::1424380312
+3::87::2::1424380312
+3::88::4::1424380312
+3::89::1::1424380312
+3::91::1::1424380312
+3::94::3::1424380312
+4::1::1::1424380312
+4::6::1::1424380312
+4::8::1::1424380312
+4::9::1::1424380312
+4::10::1::1424380312
+4::11::1::1424380312
+4::12::1::1424380312
+4::13::1::1424380312
+4::14::2::1424380312
+4::15::1::1424380312
+4::17::1::1424380312
+4::20::1::1424380312
+4::22::1::1424380312
+4::23::1::1424380312
+4::24::1::1424380312
+4::29::4::1424380312
+4::30::1::1424380312
+4::31::1::1424380312
+4::34::1::1424380312
+4::35::1::1424380312
+4::36::1::1424380312
+4::39::2::1424380312
+4::40::3::1424380312
+4::41::4::1424380312
+4::43::2::1424380312
+4::44::1::1424380312
+4::45::1::1424380312
+4::46::1::1424380312
+4::47::1::1424380312
+4::49::2::1424380312
+4::50::1::1424380312
+4::51::1::1424380312
+4::52::4::1424380312
+4::54::1::1424380312
+4::55::1::1424380312
+4::60::3::1424380312
+4::61::1::1424380312
+4::62::4::1424380312
+4::63::3::1424380312
+4::65::1::1424380312
+4::67::2::1424380312
+4::69::1::1424380312
+4::70::4::1424380312
+4::71::1::1424380312
+4::73::1::1424380312
+4::78::1::1424380312
+4::84::1::1424380312
+4::85::1::1424380312
+4::87::3::1424380312
+4::88::3::1424380312
+4::89::2::1424380312
+4::96::1::1424380312
+4::97::1::1424380312
+4::98::1::1424380312
+4::99::1::1424380312
+5::0::1::1424380312
+5::1::1::1424380312
+5::4::1::1424380312
+5::5::1::1424380312
+5::8::1::1424380312
+5::9::3::1424380312
+5::10::2::1424380312
+5::13::3::1424380312
+5::15::1::1424380312
+5::19::1::1424380312
+5::20::3::1424380312
+5::21::2::1424380312
+5::23::3::1424380312
+5::27::1::1424380312
+5::28::1::1424380312
+5::29::1::1424380312
+5::31::1::1424380312
+5::36::3::1424380312
+5::38::2::1424380312
+5::39::1::1424380312
+5::42::1::1424380312
+5::48::3::1424380312
+5::49::4::1424380312
+5::50::3::1424380312
+5::51::1::1424380312
+5::52::1::1424380312
+5::54::1::1424380312
+5::55::5::1424380312
+5::56::3::1424380312
+5::58::1::1424380312
+5::60::1::1424380312
+5::61::1::1424380312
+5::64::3::1424380312
+5::65::2::1424380312
+5::68::4::1424380312
+5::70::1::1424380312
+5::71::1::1424380312
+5::72::1::1424380312
+5::74::1::1424380312
+5::79::1::1424380312
+5::81::2::1424380312
+5::84::1::1424380312
+5::85::1::1424380312
+5::86::1::1424380312
+5::88::1::1424380312
+5::90::4::1424380312
+5::91::2::1424380312
+5::95::2::1424380312
+5::99::1::1424380312
+6::0::1::1424380312
+6::1::1::1424380312
+6::2::3::1424380312
+6::5::1::1424380312
+6::6::1::1424380312
+6::9::1::1424380312
+6::10::1::1424380312
+6::15::2::1424380312
+6::16::2::1424380312
+6::17::1::1424380312
+6::18::1::1424380312
+6::20::1::1424380312
+6::21::1::1424380312
+6::22::1::1424380312
+6::24::1::1424380312
+6::25::5::1424380312
+6::26::1::1424380312
+6::28::1::1424380312
+6::30::1::1424380312
+6::33::1::1424380312
+6::38::1::1424380312
+6::39::1::1424380312
+6::43::4::1424380312
+6::44::1::1424380312
+6::45::1::1424380312
+6::48::1::1424380312
+6::49::1::1424380312
+6::50::1::1424380312
+6::53::1::1424380312
+6::54::1::1424380312
+6::55::1::1424380312
+6::56::1::1424380312
+6::58::4::1424380312
+6::59::1::1424380312
+6::60::1::1424380312
+6::61::3::1424380312
+6::63::3::1424380312
+6::66::1::1424380312
+6::67::3::1424380312
+6::68::1::1424380312
+6::69::1::1424380312
+6::71::2::1424380312
+6::73::1::1424380312
+6::75::1::1424380312
+6::77::1::1424380312
+6::79::1::1424380312
+6::81::1::1424380312
+6::84::1::1424380312
+6::85::3::1424380312
+6::86::1::1424380312
+6::87::1::1424380312
+6::88::1::1424380312
+6::89::1::1424380312
+6::91::2::1424380312
+6::94::1::1424380312
+6::95::2::1424380312
+6::96::1::1424380312
+7::1::1::1424380312
+7::2::2::1424380312
+7::3::1::1424380312
+7::4::1::1424380312
+7::7::1::1424380312
+7::10::1::1424380312
+7::11::2::1424380312
+7::14::2::1424380312
+7::15::1::1424380312
+7::16::1::1424380312
+7::18::1::1424380312
+7::21::1::1424380312
+7::22::1::1424380312
+7::23::1::1424380312
+7::25::5::1424380312
+7::26::1::1424380312
+7::29::4::1424380312
+7::30::1::1424380312
+7::31::3::1424380312
+7::32::1::1424380312
+7::33::1::1424380312
+7::35::1::1424380312
+7::37::2::1424380312
+7::39::3::1424380312
+7::40::2::1424380312
+7::42::2::1424380312
+7::44::1::1424380312
+7::45::2::1424380312
+7::47::4::1424380312
+7::48::1::1424380312
+7::49::1::1424380312
+7::53::1::1424380312
+7::54::1::1424380312
+7::55::1::1424380312
+7::56::1::1424380312
+7::59::1::1424380312
+7::61::2::1424380312
+7::62::3::1424380312
+7::63::2::1424380312
+7::66::1::1424380312
+7::67::3::1424380312
+7::74::1::1424380312
+7::75::1::1424380312
+7::76::3::1424380312
+7::77::1::1424380312
+7::81::1::1424380312
+7::82::1::1424380312
+7::84::2::1424380312
+7::85::4::1424380312
+7::86::1::1424380312
+7::92::2::1424380312
+7::96::1::1424380312
+7::97::1::1424380312
+7::98::1::1424380312
+8::0::1::1424380312
+8::2::4::1424380312
+8::3::2::1424380312
+8::4::2::1424380312
+8::5::1::1424380312
+8::7::1::1424380312
+8::9::1::1424380312
+8::11::1::1424380312
+8::15::1::1424380312
+8::18::1::1424380312
+8::19::1::1424380312
+8::21::1::1424380312
+8::29::5::1424380312
+8::31::3::1424380312
+8::33::1::1424380312
+8::35::1::1424380312
+8::36::1::1424380312
+8::40::2::1424380312
+8::44::1::1424380312
+8::45::1::1424380312
+8::50::1::1424380312
+8::51::1::1424380312
+8::52::5::1424380312
+8::53::5::1424380312
+8::54::1::1424380312
+8::55::1::1424380312
+8::56::1::1424380312
+8::58::4::1424380312
+8::60::3::1424380312
+8::62::4::1424380312
+8::64::1::1424380312
+8::67::3::1424380312
+8::69::1::1424380312
+8::71::1::1424380312
+8::72::3::1424380312
+8::77::3::1424380312
+8::78::1::1424380312
+8::79::1::1424380312
+8::83::1::1424380312
+8::85::5::1424380312
+8::86::1::1424380312
+8::88::1::1424380312
+8::90::1::1424380312
+8::92::2::1424380312
+8::95::4::1424380312
+8::96::3::1424380312
+8::97::1::1424380312
+8::98::1::1424380312
+8::99::1::1424380312
+9::2::3::1424380312
+9::3::1::1424380312
+9::4::1::1424380312
+9::5::1::1424380312
+9::6::1::1424380312
+9::7::5::1424380312
+9::9::1::1424380312
+9::12::1::1424380312
+9::14::3::1424380312
+9::15::1::1424380312
+9::19::1::1424380312
+9::21::1::1424380312
+9::22::1::1424380312
+9::24::1::1424380312
+9::25::1::1424380312
+9::26::1::1424380312
+9::30::3::1424380312
+9::32::4::1424380312
+9::35::2::1424380312
+9::36::2::1424380312
+9::37::2::1424380312
+9::38::1::1424380312
+9::39::1::1424380312
+9::43::3::1424380312
+9::49::5::1424380312
+9::50::3::1424380312
+9::53::1::1424380312
+9::54::1::1424380312
+9::58::1::1424380312
+9::59::1::1424380312
+9::60::1::1424380312
+9::61::1::1424380312
+9::63::3::1424380312
+9::64::3::1424380312
+9::68::1::1424380312
+9::69::1::1424380312
+9::70::3::1424380312
+9::71::1::1424380312
+9::73::2::1424380312
+9::75::1::1424380312
+9::77::2::1424380312
+9::81::2::1424380312
+9::82::1::1424380312
+9::83::1::1424380312
+9::84::1::1424380312
+9::86::1::1424380312
+9::87::4::1424380312
+9::88::1::1424380312
+9::90::3::1424380312
+9::94::2::1424380312
+9::95::3::1424380312
+9::97::2::1424380312
+9::98::1::1424380312
+10::0::3::1424380312
+10::2::4::1424380312
+10::4::3::1424380312
+10::7::1::1424380312
+10::8::1::1424380312
+10::10::1::1424380312
+10::13::2::1424380312
+10::14::1::1424380312
+10::16::2::1424380312
+10::17::1::1424380312
+10::18::1::1424380312
+10::21::1::1424380312
+10::22::1::1424380312
+10::24::1::1424380312
+10::25::3::1424380312
+10::28::1::1424380312
+10::35::1::1424380312
+10::36::1::1424380312
+10::37::1::1424380312
+10::38::1::1424380312
+10::39::1::1424380312
+10::40::4::1424380312
+10::41::2::1424380312
+10::42::3::1424380312
+10::43::1::1424380312
+10::49::3::1424380312
+10::50::1::1424380312
+10::51::1::1424380312
+10::52::1::1424380312
+10::55::2::1424380312
+10::56::1::1424380312
+10::58::1::1424380312
+10::63::1::1424380312
+10::66::1::1424380312
+10::67::2::1424380312
+10::68::1::1424380312
+10::75::1::1424380312
+10::77::1::1424380312
+10::79::1::1424380312
+10::86::1::1424380312
+10::89::3::1424380312
+10::90::1::1424380312
+10::97::1::1424380312
+10::98::1::1424380312
+11::0::1::1424380312
+11::6::2::1424380312
+11::9::1::1424380312
+11::10::1::1424380312
+11::11::1::1424380312
+11::12::1::1424380312
+11::13::4::1424380312
+11::16::1::1424380312
+11::18::5::1424380312
+11::19::4::1424380312
+11::20::1::1424380312
+11::21::1::1424380312
+11::22::1::1424380312
+11::23::5::1424380312
+11::25::1::1424380312
+11::27::5::1424380312
+11::30::5::1424380312
+11::32::5::1424380312
+11::35::3::1424380312
+11::36::2::1424380312
+11::37::2::1424380312
+11::38::4::1424380312
+11::39::1::1424380312
+11::40::1::1424380312
+11::41::1::1424380312
+11::43::2::1424380312
+11::45::1::1424380312
+11::47::1::1424380312
+11::48::5::1424380312
+11::50::4::1424380312
+11::51::3::1424380312
+11::59::1::1424380312
+11::61::1::1424380312
+11::62::1::1424380312
+11::64::1::1424380312
+11::66::4::1424380312
+11::67::1::1424380312
+11::69::5::1424380312
+11::70::1::1424380312
+11::71::3::1424380312
+11::72::3::1424380312
+11::75::3::1424380312
+11::76::1::1424380312
+11::77::1::1424380312
+11::78::1::1424380312
+11::79::5::1424380312
+11::80::3::1424380312
+11::81::4::1424380312
+11::82::1::1424380312
+11::86::1::1424380312
+11::88::1::1424380312
+11::89::1::1424380312
+11::90::4::1424380312
+11::94::2::1424380312
+11::97::3::1424380312
+11::99::1::1424380312
+12::2::1::1424380312
+12::4::1::1424380312
+12::6::1::1424380312
+12::7::3::1424380312
+12::8::1::1424380312
+12::14::1::1424380312
+12::15::2::1424380312
+12::16::4::1424380312
+12::17::5::1424380312
+12::18::2::1424380312
+12::21::1::1424380312
+12::22::2::1424380312
+12::23::3::1424380312
+12::24::1::1424380312
+12::25::1::1424380312
+12::27::5::1424380312
+12::30::2::1424380312
+12::31::4::1424380312
+12::35::5::1424380312
+12::38::1::1424380312
+12::41::1::1424380312
+12::44::2::1424380312
+12::45::1::1424380312
+12::50::4::1424380312
+12::51::1::1424380312
+12::52::1::1424380312
+12::53::1::1424380312
+12::54::1::1424380312
+12::56::2::1424380312
+12::57::1::1424380312
+12::60::1::1424380312
+12::63::1::1424380312
+12::64::5::1424380312
+12::66::3::1424380312
+12::67::1::1424380312
+12::70::1::1424380312
+12::72::1::1424380312
+12::74::1::1424380312
+12::75::1::1424380312
+12::77::1::1424380312
+12::78::1::1424380312
+12::79::3::1424380312
+12::82::2::1424380312
+12::83::1::1424380312
+12::84::1::1424380312
+12::85::1::1424380312
+12::86::1::1424380312
+12::87::1::1424380312
+12::88::1::1424380312
+12::91::3::1424380312
+12::92::1::1424380312
+12::94::4::1424380312
+12::95::2::1424380312
+12::96::1::1424380312
+12::98::2::1424380312
+13::0::1::1424380312
+13::3::1::1424380312
+13::4::2::1424380312
+13::5::1::1424380312
+13::6::1::1424380312
+13::12::1::1424380312
+13::14::2::1424380312
+13::15::1::1424380312
+13::17::1::1424380312
+13::18::3::1424380312
+13::20::1::1424380312
+13::21::1::1424380312
+13::22::1::1424380312
+13::26::1::1424380312
+13::27::1::1424380312
+13::29::3::1424380312
+13::31::1::1424380312
+13::33::1::1424380312
+13::40::2::1424380312
+13::43::2::1424380312
+13::44::1::1424380312
+13::45::1::1424380312
+13::49::1::1424380312
+13::51::1::1424380312
+13::52::2::1424380312
+13::53::3::1424380312
+13::54::1::1424380312
+13::62::1::1424380312
+13::63::2::1424380312
+13::64::1::1424380312
+13::68::1::1424380312
+13::71::1::1424380312
+13::72::3::1424380312
+13::73::1::1424380312
+13::74::3::1424380312
+13::77::2::1424380312
+13::78::1::1424380312
+13::79::2::1424380312
+13::83::3::1424380312
+13::85::1::1424380312
+13::86::1::1424380312
+13::87::2::1424380312
+13::88::2::1424380312
+13::90::1::1424380312
+13::93::4::1424380312
+13::94::1::1424380312
+13::98::1::1424380312
+13::99::1::1424380312
+14::1::1::1424380312
+14::3::3::1424380312
+14::4::1::1424380312
+14::5::1::1424380312
+14::6::1::1424380312
+14::7::1::1424380312
+14::9::1::1424380312
+14::10::1::1424380312
+14::11::1::1424380312
+14::12::1::1424380312
+14::13::1::1424380312
+14::14::3::1424380312
+14::15::1::1424380312
+14::16::1::1424380312
+14::17::1::1424380312
+14::20::1::1424380312
+14::21::1::1424380312
+14::24::1::1424380312
+14::25::2::1424380312
+14::27::1::1424380312
+14::28::1::1424380312
+14::29::5::1424380312
+14::31::3::1424380312
+14::34::1::1424380312
+14::36::1::1424380312
+14::37::2::1424380312
+14::39::2::1424380312
+14::40::1::1424380312
+14::44::1::1424380312
+14::45::1::1424380312
+14::47::3::1424380312
+14::48::1::1424380312
+14::49::1::1424380312
+14::51::1::1424380312
+14::52::5::1424380312
+14::53::3::1424380312
+14::54::1::1424380312
+14::55::1::1424380312
+14::56::1::1424380312
+14::62::4::1424380312
+14::63::5::1424380312
+14::67::3::1424380312
+14::68::1::1424380312
+14::69::3::1424380312
+14::71::1::1424380312
+14::72::4::1424380312
+14::73::1::1424380312
+14::76::5::1424380312
+14::79::1::1424380312
+14::82::1::1424380312
+14::83::1::1424380312
+14::88::1::1424380312
+14::93::3::1424380312
+14::94::1::1424380312
+14::95::2::1424380312
+14::96::4::1424380312
+14::98::1::1424380312
+15::0::1::1424380312
+15::1::4::1424380312
+15::2::1::1424380312
+15::5::2::1424380312
+15::6::1::1424380312
+15::7::1::1424380312
+15::13::1::1424380312
+15::14::1::1424380312
+15::15::1::1424380312
+15::17::2::1424380312
+15::19::2::1424380312
+15::22::2::1424380312
+15::23::2::1424380312
+15::25::1::1424380312
+15::26::3::1424380312
+15::27::1::1424380312
+15::28::2::1424380312
+15::29::1::1424380312
+15::32::1::1424380312
+15::33::2::1424380312
+15::34::1::1424380312
+15::35::2::1424380312
+15::36::1::1424380312
+15::37::1::1424380312
+15::39::1::1424380312
+15::42::1::1424380312
+15::46::5::1424380312
+15::48::2::1424380312
+15::50::2::1424380312
+15::51::1::1424380312
+15::52::1::1424380312
+15::58::1::1424380312
+15::62::1::1424380312
+15::64::3::1424380312
+15::65::2::1424380312
+15::72::1::1424380312
+15::73::1::1424380312
+15::74::1::1424380312
+15::79::1::1424380312
+15::80::1::1424380312
+15::81::1::1424380312
+15::82::2::1424380312
+15::85::1::1424380312
+15::87::1::1424380312
+15::91::2::1424380312
+15::96::1::1424380312
+15::97::1::1424380312
+15::98::3::1424380312
+16::2::1::1424380312
+16::5::3::1424380312
+16::6::2::1424380312
+16::7::1::1424380312
+16::9::1::1424380312
+16::12::1::1424380312
+16::14::1::1424380312
+16::15::1::1424380312
+16::19::1::1424380312
+16::21::2::1424380312
+16::29::4::1424380312
+16::30::2::1424380312
+16::32::1::1424380312
+16::34::1::1424380312
+16::36::1::1424380312
+16::38::1::1424380312
+16::46::1::1424380312
+16::47::3::1424380312
+16::48::1::1424380312
+16::49::1::1424380312
+16::50::1::1424380312
+16::51::5::1424380312
+16::54::5::1424380312
+16::55::1::1424380312
+16::56::2::1424380312
+16::57::1::1424380312
+16::60::1::1424380312
+16::63::2::1424380312
+16::65::1::1424380312
+16::67::1::1424380312
+16::72::1::1424380312
+16::74::1::1424380312
+16::80::1::1424380312
+16::81::1::1424380312
+16::82::1::1424380312
+16::85::5::1424380312
+16::86::1::1424380312
+16::90::5::1424380312
+16::91::1::1424380312
+16::93::1::1424380312
+16::94::3::1424380312
+16::95::2::1424380312
+16::96::3::1424380312
+16::98::3::1424380312
+16::99::1::1424380312
+17::2::1::1424380312
+17::3::1::1424380312
+17::6::1::1424380312
+17::10::4::1424380312
+17::11::1::1424380312
+17::13::2::1424380312
+17::17::5::1424380312
+17::19::1::1424380312
+17::20::5::1424380312
+17::22::4::1424380312
+17::28::1::1424380312
+17::29::1::1424380312
+17::33::1::1424380312
+17::34::1::1424380312
+17::35::2::1424380312
+17::37::1::1424380312
+17::38::1::1424380312
+17::45::1::1424380312
+17::46::5::1424380312
+17::47::1::1424380312
+17::49::3::1424380312
+17::51::1::1424380312
+17::55::5::1424380312
+17::56::3::1424380312
+17::57::1::1424380312
+17::58::1::1424380312
+17::59::1::1424380312
+17::60::1::1424380312
+17::63::1::1424380312
+17::66::1::1424380312
+17::68::4::1424380312
+17::69::1::1424380312
+17::70::1::1424380312
+17::72::1::1424380312
+17::73::3::1424380312
+17::78::1::1424380312
+17::79::1::1424380312
+17::82::2::1424380312
+17::84::1::1424380312
+17::90::5::1424380312
+17::91::3::1424380312
+17::92::1::1424380312
+17::93::1::1424380312
+17::94::4::1424380312
+17::95::2::1424380312
+17::97::1::1424380312
+18::1::1::1424380312
+18::4::3::1424380312
+18::5::2::1424380312
+18::6::1::1424380312
+18::7::1::1424380312
+18::10::1::1424380312
+18::11::4::1424380312
+18::12::2::1424380312
+18::13::1::1424380312
+18::15::1::1424380312
+18::18::1::1424380312
+18::20::1::1424380312
+18::21::2::1424380312
+18::22::1::1424380312
+18::23::2::1424380312
+18::25::1::1424380312
+18::26::1::1424380312
+18::27::1::1424380312
+18::28::5::1424380312
+18::29::1::1424380312
+18::31::1::1424380312
+18::32::1::1424380312
+18::36::1::1424380312
+18::38::5::1424380312
+18::39::5::1424380312
+18::40::1::1424380312
+18::42::1::1424380312
+18::43::1::1424380312
+18::44::4::1424380312
+18::46::1::1424380312
+18::47::1::1424380312
+18::48::1::1424380312
+18::51::2::1424380312
+18::55::1::1424380312
+18::56::1::1424380312
+18::57::1::1424380312
+18::62::1::1424380312
+18::63::1::1424380312
+18::66::3::1424380312
+18::67::1::1424380312
+18::70::1::1424380312
+18::75::1::1424380312
+18::76::3::1424380312
+18::77::1::1424380312
+18::80::3::1424380312
+18::81::3::1424380312
+18::82::1::1424380312
+18::83::5::1424380312
+18::84::1::1424380312
+18::97::1::1424380312
+18::98::1::1424380312
+18::99::2::1424380312
+19::0::1::1424380312
+19::1::1::1424380312
+19::2::1::1424380312
+19::4::1::1424380312
+19::6::2::1424380312
+19::11::1::1424380312
+19::12::1::1424380312
+19::14::1::1424380312
+19::23::1::1424380312
+19::26::1::1424380312
+19::31::1::1424380312
+19::32::4::1424380312
+19::33::1::1424380312
+19::34::1::1424380312
+19::37::1::1424380312
+19::38::1::1424380312
+19::41::1::1424380312
+19::43::1::1424380312
+19::45::1::1424380312
+19::48::1::1424380312
+19::49::1::1424380312
+19::50::2::1424380312
+19::53::2::1424380312
+19::54::3::1424380312
+19::55::1::1424380312
+19::56::2::1424380312
+19::58::1::1424380312
+19::61::1::1424380312
+19::62::1::1424380312
+19::63::1::1424380312
+19::64::1::1424380312
+19::65::1::1424380312
+19::69::2::1424380312
+19::72::1::1424380312
+19::74::3::1424380312
+19::76::1::1424380312
+19::78::1::1424380312
+19::79::1::1424380312
+19::81::1::1424380312
+19::82::1::1424380312
+19::84::1::1424380312
+19::86::1::1424380312
+19::87::2::1424380312
+19::90::4::1424380312
+19::93::1::1424380312
+19::94::4::1424380312
+19::95::2::1424380312
+19::96::1::1424380312
+19::98::4::1424380312
+20::0::1::1424380312
+20::1::1::1424380312
+20::2::2::1424380312
+20::4::2::1424380312
+20::6::1::1424380312
+20::8::1::1424380312
+20::12::1::1424380312
+20::21::2::1424380312
+20::22::5::1424380312
+20::24::2::1424380312
+20::25::1::1424380312
+20::26::1::1424380312
+20::29::2::1424380312
+20::30::2::1424380312
+20::32::2::1424380312
+20::39::1::1424380312
+20::40::1::1424380312
+20::41::2::1424380312
+20::45::2::1424380312
+20::48::1::1424380312
+20::50::1::1424380312
+20::51::3::1424380312
+20::53::3::1424380312
+20::55::1::1424380312
+20::57::2::1424380312
+20::60::1::1424380312
+20::61::1::1424380312
+20::64::1::1424380312
+20::66::1::1424380312
+20::70::2::1424380312
+20::72::1::1424380312
+20::73::2::1424380312
+20::75::4::1424380312
+20::76::1::1424380312
+20::77::4::1424380312
+20::78::1::1424380312
+20::79::1::1424380312
+20::84::2::1424380312
+20::85::2::1424380312
+20::88::3::1424380312
+20::89::1::1424380312
+20::90::3::1424380312
+20::91::1::1424380312
+20::92::2::1424380312
+20::93::1::1424380312
+20::94::4::1424380312
+20::97::1::1424380312
+21::0::1::1424380312
+21::2::4::1424380312
+21::3::1::1424380312
+21::7::2::1424380312
+21::11::1::1424380312
+21::12::1::1424380312
+21::13::1::1424380312
+21::14::3::1424380312
+21::17::1::1424380312
+21::19::1::1424380312
+21::20::1::1424380312
+21::21::1::1424380312
+21::22::1::1424380312
+21::23::1::1424380312
+21::24::1::1424380312
+21::27::1::1424380312
+21::29::5::1424380312
+21::30::2::1424380312
+21::38::1::1424380312
+21::40::2::1424380312
+21::43::3::1424380312
+21::44::1::1424380312
+21::45::1::1424380312
+21::46::1::1424380312
+21::48::1::1424380312
+21::51::1::1424380312
+21::53::5::1424380312
+21::54::1::1424380312
+21::55::1::1424380312
+21::56::1::1424380312
+21::58::3::1424380312
+21::59::3::1424380312
+21::64::1::1424380312
+21::66::1::1424380312
+21::68::1::1424380312
+21::71::1::1424380312
+21::73::1::1424380312
+21::74::4::1424380312
+21::80::1::1424380312
+21::81::1::1424380312
+21::83::1::1424380312
+21::84::1::1424380312
+21::85::3::1424380312
+21::87::4::1424380312
+21::89::2::1424380312
+21::92::2::1424380312
+21::96::3::1424380312
+21::99::1::1424380312
+22::0::1::1424380312
+22::3::2::1424380312
+22::5::2::1424380312
+22::6::2::1424380312
+22::9::1::1424380312
+22::10::1::1424380312
+22::11::1::1424380312
+22::13::1::1424380312
+22::14::1::1424380312
+22::16::1::1424380312
+22::18::3::1424380312
+22::19::1::1424380312
+22::22::5::1424380312
+22::25::1::1424380312
+22::26::1::1424380312
+22::29::3::1424380312
+22::30::5::1424380312
+22::32::4::1424380312
+22::33::1::1424380312
+22::35::1::1424380312
+22::36::3::1424380312
+22::37::1::1424380312
+22::40::1::1424380312
+22::41::3::1424380312
+22::44::1::1424380312
+22::45::2::1424380312
+22::48::1::1424380312
+22::51::5::1424380312
+22::55::1::1424380312
+22::56::2::1424380312
+22::60::3::1424380312
+22::61::1::1424380312
+22::62::4::1424380312
+22::63::1::1424380312
+22::65::1::1424380312
+22::66::1::1424380312
+22::68::4::1424380312
+22::69::4::1424380312
+22::70::3::1424380312
+22::71::1::1424380312
+22::74::5::1424380312
+22::75::5::1424380312
+22::78::1::1424380312
+22::80::3::1424380312
+22::81::1::1424380312
+22::82::1::1424380312
+22::84::1::1424380312
+22::86::1::1424380312
+22::87::3::1424380312
+22::88::5::1424380312
+22::90::2::1424380312
+22::92::3::1424380312
+22::95::2::1424380312
+22::96::2::1424380312
+22::98::4::1424380312
+22::99::1::1424380312
+23::0::1::1424380312
+23::2::1::1424380312
+23::4::1::1424380312
+23::6::2::1424380312
+23::10::4::1424380312
+23::12::1::1424380312
+23::13::4::1424380312
+23::14::1::1424380312
+23::15::1::1424380312
+23::18::4::1424380312
+23::22::2::1424380312
+23::23::4::1424380312
+23::24::1::1424380312
+23::25::1::1424380312
+23::26::1::1424380312
+23::27::5::1424380312
+23::28::1::1424380312
+23::29::1::1424380312
+23::30::4::1424380312
+23::32::5::1424380312
+23::33::2::1424380312
+23::36::3::1424380312
+23::37::1::1424380312
+23::38::1::1424380312
+23::39::1::1424380312
+23::43::1::1424380312
+23::48::5::1424380312
+23::49::5::1424380312
+23::50::4::1424380312
+23::53::1::1424380312
+23::55::5::1424380312
+23::57::1::1424380312
+23::59::1::1424380312
+23::60::1::1424380312
+23::61::1::1424380312
+23::64::4::1424380312
+23::65::5::1424380312
+23::66::2::1424380312
+23::67::1::1424380312
+23::68::3::1424380312
+23::69::1::1424380312
+23::72::1::1424380312
+23::73::3::1424380312
+23::77::1::1424380312
+23::82::2::1424380312
+23::83::1::1424380312
+23::84::1::1424380312
+23::85::1::1424380312
+23::87::3::1424380312
+23::88::1::1424380312
+23::95::2::1424380312
+23::97::1::1424380312
+24::4::1::1424380312
+24::6::3::1424380312
+24::7::1::1424380312
+24::10::2::1424380312
+24::12::1::1424380312
+24::15::1::1424380312
+24::19::1::1424380312
+24::24::1::1424380312
+24::27::3::1424380312
+24::30::5::1424380312
+24::31::1::1424380312
+24::32::3::1424380312
+24::33::1::1424380312
+24::37::1::1424380312
+24::39::1::1424380312
+24::40::1::1424380312
+24::42::1::1424380312
+24::43::3::1424380312
+24::45::2::1424380312
+24::46::1::1424380312
+24::47::1::1424380312
+24::48::1::1424380312
+24::49::1::1424380312
+24::50::1::1424380312
+24::52::5::1424380312
+24::57::1::1424380312
+24::59::4::1424380312
+24::63::4::1424380312
+24::65::1::1424380312
+24::66::1::1424380312
+24::67::1::1424380312
+24::68::3::1424380312
+24::69::5::1424380312
+24::71::1::1424380312
+24::72::4::1424380312
+24::77::4::1424380312
+24::78::1::1424380312
+24::80::1::1424380312
+24::82::1::1424380312
+24::84::1::1424380312
+24::86::1::1424380312
+24::87::1::1424380312
+24::88::2::1424380312
+24::89::1::1424380312
+24::90::5::1424380312
+24::91::1::1424380312
+24::92::1::1424380312
+24::94::2::1424380312
+24::95::1::1424380312
+24::96::5::1424380312
+24::98::1::1424380312
+24::99::1::1424380312
+25::1::3::1424380312
+25::2::1::1424380312
+25::7::1::1424380312
+25::9::1::1424380312
+25::12::3::1424380312
+25::16::3::1424380312
+25::17::1::1424380312
+25::18::1::1424380312
+25::20::1::1424380312
+25::22::1::1424380312
+25::23::1::1424380312
+25::26::2::1424380312
+25::29::1::1424380312
+25::30::1::1424380312
+25::31::2::1424380312
+25::33::4::1424380312
+25::34::3::1424380312
+25::35::2::1424380312
+25::36::1::1424380312
+25::37::1::1424380312
+25::40::1::1424380312
+25::41::1::1424380312
+25::43::1::1424380312
+25::47::4::1424380312
+25::50::1::1424380312
+25::51::1::1424380312
+25::53::1::1424380312
+25::56::1::1424380312
+25::58::2::1424380312
+25::64::2::1424380312
+25::67::2::1424380312
+25::68::1::1424380312
+25::70::1::1424380312
+25::71::4::1424380312
+25::73::1::1424380312
+25::74::1::1424380312
+25::76::1::1424380312
+25::79::1::1424380312
+25::82::1::1424380312
+25::84::2::1424380312
+25::85::1::1424380312
+25::91::3::1424380312
+25::92::1::1424380312
+25::94::1::1424380312
+25::95::1::1424380312
+25::97::2::1424380312
+26::0::1::1424380312
+26::1::1::1424380312
+26::2::1::1424380312
+26::3::1::1424380312
+26::4::4::1424380312
+26::5::2::1424380312
+26::6::3::1424380312
+26::7::5::1424380312
+26::13::3::1424380312
+26::14::1::1424380312
+26::16::1::1424380312
+26::18::3::1424380312
+26::20::1::1424380312
+26::21::3::1424380312
+26::22::5::1424380312
+26::23::5::1424380312
+26::24::5::1424380312
+26::27::1::1424380312
+26::31::1::1424380312
+26::35::1::1424380312
+26::36::4::1424380312
+26::40::1::1424380312
+26::44::1::1424380312
+26::45::2::1424380312
+26::47::1::1424380312
+26::48::1::1424380312
+26::49::3::1424380312
+26::50::2::1424380312
+26::52::1::1424380312
+26::54::4::1424380312
+26::55::1::1424380312
+26::57::3::1424380312
+26::58::1::1424380312
+26::61::1::1424380312
+26::62::2::1424380312
+26::66::1::1424380312
+26::68::4::1424380312
+26::71::1::1424380312
+26::73::4::1424380312
+26::76::1::1424380312
+26::81::3::1424380312
+26::85::1::1424380312
+26::86::3::1424380312
+26::88::5::1424380312
+26::91::1::1424380312
+26::94::5::1424380312
+26::95::1::1424380312
+26::96::1::1424380312
+26::97::1::1424380312
+27::0::1::1424380312
+27::9::1::1424380312
+27::10::1::1424380312
+27::18::4::1424380312
+27::19::3::1424380312
+27::20::1::1424380312
+27::22::2::1424380312
+27::24::2::1424380312
+27::25::1::1424380312
+27::27::3::1424380312
+27::28::1::1424380312
+27::29::1::1424380312
+27::31::1::1424380312
+27::33::3::1424380312
+27::40::1::1424380312
+27::42::1::1424380312
+27::43::1::1424380312
+27::44::3::1424380312
+27::45::1::1424380312
+27::51::3::1424380312
+27::52::1::1424380312
+27::55::3::1424380312
+27::57::1::1424380312
+27::59::1::1424380312
+27::60::1::1424380312
+27::61::1::1424380312
+27::64::1::1424380312
+27::66::3::1424380312
+27::68::1::1424380312
+27::70::1::1424380312
+27::71::2::1424380312
+27::72::1::1424380312
+27::75::3::1424380312
+27::78::1::1424380312
+27::80::3::1424380312
+27::82::1::1424380312
+27::83::3::1424380312
+27::86::1::1424380312
+27::87::2::1424380312
+27::90::1::1424380312
+27::91::1::1424380312
+27::92::1::1424380312
+27::93::1::1424380312
+27::94::2::1424380312
+27::95::1::1424380312
+27::98::1::1424380312
+28::0::3::1424380312
+28::1::1::1424380312
+28::2::4::1424380312
+28::3::1::1424380312
+28::6::1::1424380312
+28::7::1::1424380312
+28::12::5::1424380312
+28::13::2::1424380312
+28::14::1::1424380312
+28::15::1::1424380312
+28::17::1::1424380312
+28::19::3::1424380312
+28::20::1::1424380312
+28::23::3::1424380312
+28::24::3::1424380312
+28::27::1::1424380312
+28::29::1::1424380312
+28::33::1::1424380312
+28::34::1::1424380312
+28::36::1::1424380312
+28::38::2::1424380312
+28::39::2::1424380312
+28::44::1::1424380312
+28::45::1::1424380312
+28::49::4::1424380312
+28::50::1::1424380312
+28::52::1::1424380312
+28::54::1::1424380312
+28::56::1::1424380312
+28::57::3::1424380312
+28::58::1::1424380312
+28::59::1::1424380312
+28::60::1::1424380312
+28::62::3::1424380312
+28::63::1::1424380312
+28::65::1::1424380312
+28::75::1::1424380312
+28::78::1::1424380312
+28::81::5::1424380312
+28::82::4::1424380312
+28::83::1::1424380312
+28::85::1::1424380312
+28::88::2::1424380312
+28::89::4::1424380312
+28::90::1::1424380312
+28::92::5::1424380312
+28::94::1::1424380312
+28::95::2::1424380312
+28::98::1::1424380312
+28::99::1::1424380312
+29::3::1::1424380312
+29::4::1::1424380312
+29::5::1::1424380312
+29::7::2::1424380312
+29::9::1::1424380312
+29::10::3::1424380312
+29::11::1::1424380312
+29::13::3::1424380312
+29::14::1::1424380312
+29::15::1::1424380312
+29::17::3::1424380312
+29::19::3::1424380312
+29::22::3::1424380312
+29::23::4::1424380312
+29::25::1::1424380312
+29::29::1::1424380312
+29::31::1::1424380312
+29::32::4::1424380312
+29::33::2::1424380312
+29::36::2::1424380312
+29::38::3::1424380312
+29::39::1::1424380312
+29::42::1::1424380312
+29::46::5::1424380312
+29::49::3::1424380312
+29::51::2::1424380312
+29::59::1::1424380312
+29::61::1::1424380312
+29::62::1::1424380312
+29::67::1::1424380312
+29::68::3::1424380312
+29::69::1::1424380312
+29::70::1::1424380312
+29::74::1::1424380312
+29::75::1::1424380312
+29::79::2::1424380312
+29::80::1::1424380312
+29::81::2::1424380312
+29::83::1::1424380312
+29::85::1::1424380312
+29::86::1::1424380312
+29::90::4::1424380312
+29::93::1::1424380312
+29::94::4::1424380312
+29::97::1::1424380312
+29::99::1::1424380312
diff --git a/data/mllib/gmm_data.txt b/data/mllib/gmm_data.txt
new file mode 100644
index 0000000000000..934ee4a83a2df
--- /dev/null
+++ b/data/mllib/gmm_data.txt
@@ -0,0 +1,2000 @@
+ 2.59470454e+00 2.12298217e+00
+ 1.15807024e+00 -1.46498723e-01
+ 2.46206638e+00 6.19556894e-01
+ -5.54845070e-01 -7.24700066e-01
+ -3.23111426e+00 -1.42579084e+00
+ 3.02978115e+00 7.87121753e-01
+ 1.97365907e+00 1.15914704e+00
+ -6.44852101e+00 -3.18154314e+00
+ 1.30963349e+00 1.62866434e-01
+ 4.26482541e+00 2.15547996e+00
+ 3.79927257e+00 1.50572445e+00
+ 4.17452609e-01 -6.74032760e-01
+ 4.21117627e-01 4.45590255e-01
+ -2.80425571e+00 -7.77150554e-01
+ 2.55928797e+00 7.03954218e-01
+ 1.32554059e+00 -9.46663152e-01
+ -3.39691439e+00 -1.49005743e+00
+ -2.26542270e-01 3.60052515e-02
+ 1.04994198e+00 5.29825685e-01
+ -1.51566882e+00 -1.86264432e-01
+ -3.27928172e-01 -7.60859110e-01
+ -3.18054866e-01 3.97719805e-01
+ 1.65579418e-01 -3.47232033e-01
+ 6.47162333e-01 4.96059961e-02
+ -2.80776647e-01 4.79418757e-01
+ 7.45069752e-01 1.20790281e-01
+ 2.13604102e-01 1.59542555e-01
+ -3.08860224e+00 -1.43259870e+00
+ 8.97066497e-01 1.10206801e+00
+ -2.23918874e-01 -1.07267267e+00
+ 2.51525708e+00 2.84761973e-01
+ 9.98052532e-01 1.08333783e+00
+ 1.76705588e+00 8.18866778e-01
+ 5.31555163e-02 -1.90111151e-01
+ -2.17405059e+00 7.21854582e-02
+ -2.13772505e+00 -3.62010387e-01
+ 2.95974057e+00 1.31602381e+00
+ 2.74053561e+00 1.61781757e+00
+ 6.68135448e-01 2.86586009e-01
+ 2.82323739e+00 1.74437257e+00
+ 8.11540288e-01 5.50744478e-01
+ 4.10050897e-01 5.10668402e-03
+ 9.58626136e-01 -3.49633680e-01
+ 4.66599798e+00 1.49964894e+00
+ 4.94507794e-01 2.58928077e-01
+ -2.36029742e+00 -1.61042909e+00
+ -4.99306804e-01 -8.04984769e-01
+ 1.07448510e+00 9.39605828e-01
+ -1.80448949e+00 -1.05983264e+00
+ -3.22353821e-01 1.73612093e-01
+ 1.85418702e+00 1.15640643e+00
+ 6.93794163e-01 6.59993560e-01
+ 1.99399102e+00 1.44547123e+00
+ 3.38866124e+00 1.23379290e+00
+ -4.24067720e+00 -1.22264282e+00
+ 6.03230201e-02 2.95232729e-01
+ -3.59341813e+00 -7.17453726e-01
+ 4.87447372e-01 -2.00733911e-01
+ 1.20149195e+00 4.07880197e-01
+ -2.13331464e+00 -4.58518077e-01
+ -3.84091083e+00 -1.71553950e+00
+ -5.37279250e-01 2.64822629e-02
+ -2.10155227e+00 -1.32558103e+00
+ -1.71318897e+00 -7.12098563e-01
+ -1.46280695e+00 -1.84868337e-01
+ -3.59785325e+00 -1.54832434e+00
+ -5.77528081e-01 -5.78580857e-01
+ 3.14734283e-01 5.80184639e-01
+ -2.71164714e+00 -1.19379432e+00
+ 1.09634489e+00 7.20143887e-01
+ -3.05527722e+00 -1.47774064e+00
+ 6.71753586e-01 7.61350020e-01
+ 3.98294144e+00 1.54166484e+00
+ -3.37220384e+00 -2.21332064e+00
+ 1.81222914e+00 7.41212752e-01
+ 2.71458282e-01 1.36329078e-01
+ -3.97815359e-01 1.16766886e-01
+ -1.70192814e+00 -9.75851571e-01
+ -3.46803804e+00 -1.09965988e+00
+ -1.69649627e+00 -5.76045801e-01
+ -1.02485636e-01 -8.81841246e-01
+ -3.24194667e-02 2.55429276e-01
+ -2.75343168e+00 -1.51366320e+00
+ -2.78676702e+00 -5.22360489e-01
+ 1.70483164e+00 1.19769805e+00
+ 4.92022579e-01 3.24944706e-01
+ 2.48768464e+00 1.00055363e+00
+ 4.48786400e-01 7.63902870e-01
+ 2.93862696e+00 1.73809968e+00
+ -3.55019305e+00 -1.97875558e+00
+ 1.74270784e+00 6.90229224e-01
+ 5.13391994e-01 4.58374016e-01
+ 1.78379499e+00 9.08026381e-01
+ 1.75814147e+00 7.41449784e-01
+ -2.30687792e-01 3.91009729e-01
+ 3.92271353e+00 1.44006290e+00
+ 2.93361679e-01 -4.99886375e-03
+ 2.47902690e-01 -7.49542503e-01
+ -3.97675355e-01 1.36824887e-01
+ 3.56535953e+00 1.15181329e+00
+ 3.22425301e+00 1.28702383e+00
+ -2.94192478e-01 -2.42382557e-01
+ 8.02068864e-01 -1.51671475e-01
+ 8.54133530e-01 -4.89514885e-02
+ -1.64316316e-01 -5.34642346e-01
+ -6.08485405e-01 -2.10332352e-01
+ -2.18940059e+00 -1.07024952e+00
+ -1.71586960e+00 -2.83333492e-02
+ 1.70200448e-01 -3.28031178e-01
+ -1.97210346e+00 -5.39948532e-01
+ 2.19500160e+00 1.05697170e+00
+ -1.76239935e+00 -1.09377438e+00
+ 1.68314744e+00 6.86491164e-01
+ -2.99852288e+00 -1.46619067e+00
+ -2.23769560e+00 -9.15008355e-01
+ 9.46887516e-01 5.58410503e-01
+ 5.02153123e-01 1.63851235e-01
+ -9.70297062e-01 3.14625374e-01
+ -1.29405593e+00 -8.20994131e-01
+ 2.72516079e+00 7.85839947e-01
+ 1.45788024e+00 3.37487353e-01
+ -4.36292749e-01 -5.42150480e-01
+ 2.21304711e+00 1.25254042e+00
+ -1.20810271e-01 4.79632898e-01
+ -3.30884511e+00 -1.50607586e+00
+ -6.55882455e+00 -1.94231256e+00
+ -3.17033630e+00 -9.94678930e-01
+ 1.42043617e+00 7.28808957e-01
+ -1.57546099e+00 -1.10320497e+00
+ -3.22748754e+00 -1.64174579e+00
+ 2.96776017e-03 -3.16191512e-02
+ -2.25986054e+00 -6.13123197e-01
+ 2.49434243e+00 7.73069183e-01
+ 9.08494049e-01 -1.53926853e-01
+ -2.80559090e+00 -1.37474221e+00
+ 4.75224286e-01 2.53153674e-01
+ 4.37644006e+00 8.49116998e-01
+ 2.27282959e+00 6.16568202e-01
+ 1.16006880e+00 1.65832798e-01
+ -1.67163193e+00 -1.22555386e+00
+ -1.38231118e+00 -7.29575504e-01
+ -3.49922750e+00 -2.26446675e+00
+ -3.73780110e-01 -1.90657869e-01
+ 1.68627679e+00 1.05662987e+00
+ -3.28891792e+00 -1.11080334e+00
+ -2.59815798e+00 -1.51410198e+00
+ -2.61203309e+00 -6.00143552e-01
+ 6.58964943e-01 4.47216094e-01
+ -2.26711381e+00 -7.26512923e-01
+ -5.31429009e-02 -1.97925341e-02
+ 3.19749807e+00 9.20425476e-01
+ -1.37595787e+00 -6.58062732e-01
+ 8.09900278e-01 -3.84286160e-01
+ -5.07741280e+00 -1.97683808e+00
+ -2.99764250e+00 -1.50753777e+00
+ -9.87671815e-01 -4.63255889e-01
+ 1.65390765e+00 6.73806615e-02
+ 5.51252659e+00 2.69842267e+00
+ -2.23724309e+00 -4.77624004e-01
+ 4.99726228e+00 1.74690949e+00
+ 1.75859162e-01 -1.49350995e-01
+ 4.13382789e+00 1.31735161e+00
+ 2.69058117e+00 4.87656923e-01
+ 1.07180318e+00 1.01426954e+00
+ 3.37216869e+00 1.05955377e+00
+ -2.95006781e+00 -1.57048303e+00
+ -2.46401648e+00 -8.37056374e-01
+ 1.19012962e-01 7.54702770e-01
+ 3.34142539e+00 4.81938295e-01
+ 2.92643913e+00 1.04301050e+00
+ 2.89697751e+00 1.37551442e+00
+ -1.03094242e+00 2.20903962e-01
+ -5.13914589e+00 -2.23355387e+00
+ -8.81680780e-01 1.83590000e-01
+ 2.82334775e+00 1.26650464e+00
+ -2.81042540e-01 -3.26370240e-01
+ 2.97995487e+00 8.34569452e-01
+ -1.39857135e+00 -1.15798385e+00
+ 4.27186506e+00 9.04253702e-01
+ 6.98684517e-01 7.91167305e-01
+ 3.52233095e+00 1.29976473e+00
+ 2.21448029e+00 2.73213379e-01
+ -3.13505683e-01 -1.20593774e-01
+ 3.70571571e+00 1.06220876e+00
+ 9.83881041e-01 5.67713803e-01
+ -2.17897705e+00 2.52925205e-01
+ 1.38734039e+00 4.61287066e-01
+ -1.41181602e+00 -1.67248955e-02
+ -1.69974639e+00 -7.17812071e-01
+ -2.01005793e-01 -7.49662056e-01
+ 1.69016336e+00 3.24687979e-01
+ -2.03250179e+00 -2.76108460e-01
+ 3.68776848e-01 4.12536941e-01
+ 7.66238259e-01 -1.84750637e-01
+ -2.73989147e-01 -1.72817250e-01
+ -2.18623745e+00 -2.10906798e-01
+ -1.39795625e-01 3.26066094e-02
+ -2.73826912e-01 -6.67586097e-02
+ -1.57880654e+00 -4.99395900e-01
+ 4.55950908e+00 2.29410489e+00
+ -7.36479631e-01 -1.57861857e-01
+ 1.92082888e+00 1.05843391e+00
+ 4.29192810e+00 1.38127810e+00
+ 1.61852879e+00 1.95871986e-01
+ -1.95027403e+00 -5.22448168e-01
+ -1.67446281e+00 -9.41497162e-01
+ 6.07097859e-01 3.44178029e-01
+ -3.44004683e+00 -1.49258461e+00
+ 2.72114752e+00 6.00728991e-01
+ 8.80685522e-01 -2.53243336e-01
+ 1.39254928e+00 3.42988512e-01
+ 1.14194836e-01 -8.57945694e-02
+ -1.49387332e+00 -7.60860481e-01
+ -1.98053285e+00 -4.86039865e-01
+ 3.56008568e+00 1.08438692e+00
+ 2.27833961e-01 1.09441881e+00
+ -1.16716710e+00 -6.54778242e-01
+ 2.02156613e+00 5.42075758e-01
+ 1.08429178e+00 -7.67420693e-01
+ 6.63058455e-01 4.61680991e-01
+ -1.06201537e+00 1.38862846e-01
+ 3.08701875e+00 8.32580273e-01
+ -4.96558108e-01 -2.47031257e-01
+ 7.95109987e-01 7.59314147e-02
+ -3.39903524e-01 8.71565566e-03
+ 8.68351357e-01 4.78358641e-01
+ 1.48750819e+00 7.63257420e-01
+ -4.51224101e-01 -4.44056898e-01
+ -3.02734750e-01 -2.98487961e-01
+ 5.46846609e-01 7.02377629e-01
+ 1.65129778e+00 3.74008231e-01
+ -7.43336512e-01 3.95723531e-01
+ -5.88446605e-01 -6.47520211e-01
+ 3.58613167e+00 1.95024937e+00
+ 3.11718883e+00 8.37984715e-01
+ 1.80919244e+00 9.62644986e-01
+ 5.43856371e-02 -5.86297543e-01
+ -1.95186766e+00 -1.02624212e-01
+ 8.95628057e-01 5.91812281e-01
+ 4.97691627e-02 5.31137156e-01
+ -1.07633113e+00 -2.47392788e-01
+ -1.17257986e+00 -8.68528265e-01
+ -8.19227665e-02 5.80579434e-03
+ -2.86409787e-01 1.95812924e-01
+ 1.10582671e+00 7.42853240e-01
+ 4.06429774e+00 1.06557476e+00
+ -3.42521792e+00 -7.74327139e-01
+ 1.28468671e+00 6.20431661e-01
+ 6.01201008e-01 -1.16799728e-01
+ -1.85058727e-01 -3.76235293e-01
+ 5.44083324e+00 2.98490868e+00
+ 2.69273070e+00 7.83901153e-01
+ 1.88938036e-01 -4.83222152e-01
+ 1.05667256e+00 -2.57003165e-01
+ 2.99711662e-01 -4.33131912e-01
+ 7.73689216e-02 -1.78738364e-01
+ 9.58326279e-01 6.38325706e-01
+ -3.97727049e-01 2.27314759e-01
+ 3.36098175e+00 1.12165237e+00
+ 1.77804871e+00 6.46961933e-01
+ -2.86945546e+00 -1.00395518e+00
+ 3.03494815e+00 7.51814612e-01
+ -1.43658194e+00 -3.55432244e-01
+ -3.08455105e+00 -1.51535106e+00
+ -1.55841975e+00 3.93454820e-02
+ 7.96073412e-01 -3.11036969e-01
+ -9.84125401e-01 -1.02064649e+00
+ -7.75688143e+00 -3.65219926e+00
+ 1.53816429e+00 7.65926670e-01
+ -4.92712738e-01 2.32244240e-02
+ -1.93166919e+00 -1.07701304e+00
+ 2.03029875e-02 -7.54055699e-01
+ 2.52177489e+00 1.01544979e+00
+ 3.65109048e-01 -9.48328494e-01
+ -1.28849143e-01 2.51947174e-01
+ -1.02428075e+00 -9.37767116e-01
+ -3.04179748e+00 -9.97926994e-01
+ -2.51986980e+00 -1.69117413e+00
+ -1.24900838e+00 -4.16179917e-01
+ 2.77943992e+00 1.22842327e+00
+ -4.37434557e+00 -1.70182693e+00
+ -1.60019319e+00 -4.18345639e-01
+ -1.67613646e+00 -9.44087262e-01
+ -9.00843245e-01 8.26378089e-02
+ 3.29770621e-01 -9.07870444e-01
+ -2.84650535e+00 -9.00155396e-01
+ 1.57111705e+00 7.07432268e-01
+ 1.24948552e+00 1.04812849e-01
+ 1.81440558e+00 9.53545082e-01
+ -1.74915794e+00 -1.04606288e+00
+ 1.20593269e+00 -1.12607147e-02
+ 1.36004919e-01 -1.09828044e+00
+ 2.57480693e-01 3.34941541e-01
+ 7.78775385e-01 -5.32494732e-01
+ -1.79155126e+00 -6.29994129e-01
+ -1.75706839e+00 -8.35100126e-01
+ 4.29512012e-01 7.81426910e-02
+ 3.08349370e-01 -1.27359861e-01
+ 1.05560329e+00 4.55150640e-01
+ 1.95662574e+00 1.17593217e+00
+ 8.77376632e-01 6.57866662e-01
+ 7.71311255e-01 9.15134334e-02
+ -6.36978275e+00 -2.55874241e+00
+ -2.98335339e+00 -1.59567024e+00
+ -3.67104587e-01 1.85315291e-01
+ 1.95347407e+00 -7.15503113e-02
+ 8.45556363e-01 6.51256415e-02
+ 9.42868521e-01 3.56647624e-01
+ 2.99321875e+00 1.07505254e+00
+ -2.91030538e-01 -3.77637183e-01
+ 1.62870918e+00 3.37563671e-01
+ 2.05773173e-01 3.43337416e-01
+ -8.40879199e-01 -1.35600767e-01
+ 1.38101624e+00 5.99253495e-01
+ -6.93715607e+00 -2.63580662e+00
+ -1.04423404e+00 -8.32865050e-01
+ 1.33448476e+00 1.04863475e+00
+ 6.01675207e-01 1.98585194e-01
+ 2.31233993e+00 7.98628331e-01
+ 1.85201313e-01 -1.76070247e+00
+ 1.92006354e+00 8.45737582e-01
+ 1.06320415e+00 2.93426068e-01
+ -1.20360141e+00 -1.00301288e+00
+ 1.95926629e+00 6.26643532e-01
+ 6.04483978e-02 5.72643059e-01
+ -1.04568563e+00 -5.91021496e-01
+ 2.62300678e+00 9.50997831e-01
+ -4.04610275e-01 3.73150879e-01
+ 2.26371902e+00 8.73627529e-01
+ 2.12545313e+00 7.90640352e-01
+ 7.72181917e-03 1.65718952e-02
+ 1.00422340e-01 -2.05562936e-01
+ -1.22989802e+00 -1.01841681e-01
+ 3.09064082e+00 1.04288010e+00
+ 5.18274167e+00 1.34749259e+00
+ -8.32075153e-01 -1.97592029e-01
+ 3.84126764e-02 5.58171345e-01
+ 4.99560727e-01 -4.26154438e-02
+ 4.79071151e+00 2.19728942e+00
+ -2.78437968e+00 -1.17812590e+00
+ -2.22804226e+00 -4.31174255e-01
+ 8.50762292e-01 -1.06445261e-01
+ 1.10812830e+00 -2.59118812e-01
+ -2.91450155e-01 6.42802679e-01
+ -1.38631532e-01 -5.88585623e-01
+ -5.04120983e-01 -2.17094915e-01
+ 3.41410820e+00 1.67897767e+00
+ -2.23697326e+00 -6.62735244e-01
+ -3.55961064e-01 -1.27647226e-01
+ -3.55568274e+00 -2.49011369e+00
+ -8.77586408e-01 -9.38268065e-03
+ 1.52382384e-01 -5.62155760e-01
+ 1.55885574e-01 1.07617069e-01
+ -8.37129973e-01 -5.22259081e-01
+ -2.92741750e+00 -1.35049428e+00
+ -3.54670781e-01 5.69205952e-02
+ 2.21030255e+00 1.34689986e+00
+ 1.60787722e+00 5.75984706e-01
+ 1.32294221e+00 5.31577509e-01
+ 7.05672928e-01 3.34241244e-01
+ 1.41406179e+00 1.15783408e+00
+ -6.92172228e-01 -2.84817896e-01
+ 3.28358655e-01 -2.66910083e-01
+ 1.68013644e-01 -4.28016549e-02
+ 2.07365974e+00 7.76496211e-01
+ -3.92974907e-01 2.46796730e-01
+ -5.76078636e-01 3.25676963e-01
+ -1.82547204e-01 -5.06410543e-01
+ 3.04754906e+00 1.16174496e+00
+ -3.01090632e+00 -1.09195183e+00
+ -1.44659696e+00 -6.87838682e-01
+ 2.11395861e+00 9.10495785e-01
+ 1.40962871e+00 1.13568678e+00
+ -1.66653234e-01 -2.10012503e-01
+ 3.17456029e+00 9.74502922e-01
+ 2.15944820e+00 8.62807189e-01
+ -3.45418719e+00 -1.33647548e+00
+ -3.41357732e+00 -8.47048920e-01
+ -3.06702448e-01 -6.64280634e-01
+ -2.86930714e-01 -1.35268264e-01
+ -3.15835557e+00 -5.43439253e-01
+ 2.49541440e-01 -4.71733570e-01
+ 2.71933912e+00 4.13308399e-01
+ -2.43787038e+00 -1.08050547e+00
+ -4.90234490e-01 -6.64069865e-01
+ 8.99524451e-02 5.76180541e-01
+ 5.00500404e+00 2.12125521e+00
+ -1.73107940e-01 -2.28506575e-02
+ 5.44938858e-01 -1.29523352e-01
+ 5.13526842e+00 1.68785993e+00
+ 1.70228304e+00 1.02601138e+00
+ 3.58957507e+00 1.54396196e+00
+ 1.85615738e+00 4.92916197e-01
+ 2.55772147e+00 7.88438908e-01
+ -1.57008279e+00 -4.17377300e-01
+ -1.42548604e+00 -3.63684860e-01
+ -8.52026118e-01 2.72052686e-01
+ -5.10563077e+00 -2.35665994e+00
+ -2.95517031e+00 -1.84945297e+00
+ -2.91947959e+00 -1.66016784e+00
+ -4.21462387e+00 -1.41131535e+00
+ 6.59901121e-01 4.87156314e-01
+ -9.75352532e-01 -4.50231285e-01
+ -5.94084444e-01 -1.16922670e+00
+ 7.50554615e-01 -9.83692552e-01
+ 1.07054926e+00 2.77143030e-01
+ -3.88079578e-01 -4.17737309e-02
+ -9.59373733e-01 -8.85454886e-01
+ -7.53560665e-02 -5.16223870e-02
+ 9.84108158e-01 -5.89290700e-02
+ 1.87272961e-01 -4.34238391e-01
+ 6.86509981e-01 -3.15116460e-01
+ -1.07762538e+00 6.58984161e-02
+ 6.09266592e-01 6.91808473e-02
+ -8.30529954e-01 -7.00454791e-01
+ -9.13179464e-01 -6.31712891e-01
+ 7.68744851e-01 1.09840676e+00
+ -1.07606690e+00 -8.78390282e-01
+ -1.71038184e+00 -5.73606033e-01
+ 8.75982765e-01 3.66343143e-01
+ -7.04919009e-01 -8.49182590e-01
+ -1.00274668e+00 -7.99573611e-01
+ -1.05562848e+00 -5.84060076e-01
+ 4.03490015e+00 1.28679206e+00
+ -3.53484804e+00 -1.71381255e+00
+ 2.31527363e-01 1.04179397e-01
+ -3.58592392e-02 3.74895739e-01
+ 3.92253428e+00 1.81852726e+00
+ -7.27384249e-01 -6.45605128e-01
+ 4.65678097e+00 2.41379899e+00
+ 1.16750534e+00 7.60718205e-01
+ 1.15677059e+00 7.96225550e-01
+ -1.42920261e+00 -4.66946295e-01
+ 3.71148192e+00 1.88060191e+00
+ 2.44052407e+00 3.84472199e-01
+ -1.64535035e+00 -8.94530036e-01
+ -3.69608753e+00 -1.36402754e+00
+ 2.24419208e+00 9.69744889e-01
+ 2.54822427e+00 1.22613039e+00
+ 3.77484909e-01 -5.98521878e-01
+ -3.61521175e+00 -1.11123912e+00
+ 3.28113127e+00 1.52551775e+00
+ -3.51030902e+00 -1.53913980e+00
+ -2.44874505e+00 -6.30246005e-01
+ -3.42516153e-01 -5.07352665e-01
+ 1.09110502e+00 6.36821628e-01
+ -2.49434967e+00 -8.02827146e-01
+ 1.41763139e+00 -3.46591820e-01
+ 1.61108619e+00 5.93871102e-01
+ 3.97371717e+00 1.35552499e+00
+ -1.33437177e+00 -2.83908670e-01
+ -1.41606483e+00 -1.76402601e-01
+ 2.23945322e-01 -1.77157065e-01
+ 2.60271569e+00 2.40778251e-01
+ -2.82213895e-02 1.98255474e-01
+ 4.20727940e+00 1.31490863e+00
+ 3.36944889e+00 1.57566635e+00
+ 3.53049396e+00 1.73579350e+00
+ -1.29170202e+00 -1.64196290e+00
+ 9.27295604e-01 9.98808036e-01
+ 1.75321843e-01 -2.83267817e-01
+ -2.19069578e+00 -1.12814358e+00
+ 1.66606031e+00 7.68006933e-01
+ -7.13826035e-01 5.20881684e-02
+ -3.43821888e+00 -2.36137021e+00
+ -5.93210310e-01 1.21843813e-01
+ -4.09800822e+00 -1.39893953e+00
+ 2.74110954e+00 1.52728606e+00
+ 1.72652512e+00 -1.25435113e-01
+ 1.97722357e+00 6.40667481e-01
+ 4.18635780e-01 3.57018509e-01
+ -1.78303569e+00 -2.11864764e-01
+ -3.52809366e+00 -2.58794450e-01
+ -4.72407090e+00 -1.63870734e+00
+ 1.73917807e+00 8.73251829e-01
+ 4.37979356e-01 8.49210569e-01
+ 3.93791881e+00 1.76269490e+00
+ 2.79065411e+00 1.04019042e+00
+ -8.47426142e-01 -3.40136892e-01
+ -4.24389181e+00 -1.80253120e+00
+ -1.86675870e+00 -7.64558265e-01
+ 9.46212675e-01 -7.77681445e-02
+ -2.82448462e+00 -1.33592449e+00
+ -2.57938567e+00 -1.56554690e+00
+ -2.71615767e+00 -6.27667233e-01
+ -1.55999166e+00 -5.81013466e-01
+ -4.24696864e-01 -7.44673250e-01
+ 1.67592970e+00 7.68164292e-01
+ 8.48455216e-01 -6.05681126e-01
+ 6.12575454e+00 1.65607584e+00
+ 1.38207327e+00 2.39261863e-01
+ 3.13364450e+00 1.17154698e+00
+ 1.71694858e+00 1.26744905e+00
+ -1.61746367e+00 -8.80098073e-01
+ -8.52196756e-01 -9.27299728e-01
+ -1.51562462e-01 -8.36552490e-02
+ -7.04792753e-01 -1.24726713e-02
+ -3.35265757e+00 -1.82176312e+00
+ 3.32173170e-01 -1.33405580e-01
+ 4.95841013e-01 4.58292712e-01
+ 1.57713955e+00 7.79272991e-01
+ 2.09743109e+00 9.23542557e-01
+ 3.90450311e-03 -8.42873164e-01
+ 2.59519038e+00 7.56479591e-01
+ -5.77643976e-01 -2.36401904e-01
+ -5.22310654e-01 1.34187830e-01
+ -2.22096086e+00 -7.75507719e-01
+ 1.35907831e+00 7.80197510e-01
+ 3.80355868e+00 1.16983476e+00
+ 3.82746596e+00 1.31417718e+00
+ 3.30451183e+00 1.55398159e+00
+ -3.42917814e-01 -8.62281222e-02
+ -2.59093020e+00 -9.29883526e-01
+ 1.40928562e+00 1.08398346e+00
+ 1.54400137e-01 3.35881092e-01
+ 1.59171586e+00 1.18855802e+00
+ -5.25164002e-01 -1.03104220e-01
+ 2.20067959e+00 1.37074713e+00
+ 6.97860830e-01 6.27718548e-01
+ -4.59743507e-01 1.36061163e-01
+ -1.04691963e-01 -2.16271727e-01
+ -1.08905573e+00 -5.95510769e-01
+ -1.00826983e+00 -5.38509162e-02
+ -3.16402719e+00 -1.33414216e+00
+ 1.47870874e-01 1.75234619e-01
+ -2.57078234e-01 7.03316889e-02
+ 1.81073945e+00 4.26901462e-01
+ 2.65476530e+00 6.74217273e-01
+ 1.27539811e+00 6.22914081e-01
+ -3.76750499e-01 -1.20629449e+00
+ 1.00177595e+00 -1.40660091e-01
+ -2.98919265e+00 -1.65145013e+00
+ -2.21557682e+00 -8.11123452e-01
+ -3.22635378e+00 -1.65639056e+00
+ -2.72868553e+00 -1.02812087e+00
+ 1.26042797e+00 8.49005248e-01
+ -9.38318534e-01 -9.87588651e-01
+ 3.38013194e-01 -1.00237461e-01
+ 1.91175691e+00 8.48716369e-01
+ 4.30244344e-01 6.05539915e-02
+ 2.21783435e+00 3.03268204e-01
+ 1.78019576e+00 1.27377108e+00
+ 1.59733274e+00 4.40674687e-02
+ 3.97428484e+00 2.20881566e+00
+ -2.41108677e+00 -6.01410418e-01
+ -2.50796499e+00 -5.71169866e-01
+ -3.71957427e+00 -1.38195726e+00
+ -1.57992670e+00 1.32068593e-01
+ -1.35278851e+00 -6.39349270e-01
+ 1.23075932e+00 2.40445409e-01
+ 1.35606530e+00 4.33180078e-01
+ 9.60968518e-02 2.26734255e-01
+ 6.22975063e-01 5.03431915e-02
+ -1.47624851e+00 -3.60568238e-01
+ -2.49337808e+00 -1.15083052e+00
+ 2.15717792e+00 1.03071559e+00
+ -3.07814376e-02 1.38700314e-02
+ 4.52049499e-02 -4.86409775e-01
+ 2.58231061e+00 1.14327809e-01
+ 1.10999138e+00 -5.18568405e-01
+ -2.19426443e-01 -5.37505538e-01
+ -4.44740298e-01 6.78099955e-01
+ 4.03379080e+00 1.49825720e+00
+ -5.13182408e-01 -4.90201950e-01
+ -6.90139716e-01 1.63875126e-01
+ -8.17281461e-01 2.32155064e-01
+ -2.92357619e-01 -8.02573544e-01
+ -1.80769841e+00 -7.58907326e-01
+ 2.16981590e+00 1.06728873e+00
+ 1.98995203e-01 -6.84176682e-02
+ -2.39546753e+00 -2.92873789e-01
+ -4.24251021e+00 -1.46255564e+00
+ -5.01411291e-01 -5.95712813e-03
+ 2.68085809e+00 1.42883780e+00
+ -4.13289873e+00 -1.62729388e+00
+ 1.87957843e+00 3.63341638e-01
+ -1.15270744e+00 -3.03563774e-01
+ -4.43994248e+00 -2.97323905e+00
+ -7.17067733e-01 -7.08349542e-01
+ -3.28870393e+00 -1.19263863e+00
+ -7.55325944e-01 -5.12703329e-01
+ -2.07291938e+00 -2.65025085e-01
+ -7.50073814e-01 -1.70771041e-01
+ -8.77381404e-01 -5.47417325e-01
+ -5.33725862e-01 5.15837119e-01
+ 8.45056431e-01 2.82125560e-01
+ -1.59598637e+00 -1.38743235e+00
+ 1.41362902e+00 1.06407789e+00
+ 1.02584504e+00 -3.68219466e-01
+ -1.04644488e+00 -1.48769392e-01
+ 2.66990191e+00 8.57633492e-01
+ -1.84251857e+00 -9.82430175e-01
+ 9.71404204e-01 -2.81934209e-01
+ -2.50177989e+00 -9.21260335e-01
+ -1.31060074e+00 -5.84488113e-01
+ -2.12129400e-01 -3.06244708e-02
+ -5.28933882e+00 -2.50663129e+00
+ 1.90220541e+00 1.08662918e+00
+ -3.99366086e-02 -6.87178973e-01
+ -4.93417342e-01 4.37354182e-01
+ 2.13494486e+00 1.37679569e+00
+ 2.18396765e+00 5.81023868e-01
+ -3.07866587e+00 -1.45384974e+00
+ 6.10894119e-01 -4.17050124e-01
+ -1.88766952e+00 -8.86160058e-01
+ 3.34527253e+00 1.78571260e+00
+ 6.87769059e-01 -5.01157336e-01
+ 2.60470837e+00 1.45853560e+00
+ -6.49315691e-01 -9.16112805e-01
+ -1.29817687e+00 -2.15924339e-01
+ -1.20100409e-03 -4.03137422e-01
+ -1.36471594e+00 -6.93266356e-01
+ 1.38682062e+00 7.15131598e-01
+ 2.47830103e+00 1.24862305e+00
+ -2.78288147e+00 -1.03329235e+00
+ -7.33443403e-01 -6.11041652e-01
+ -4.12745671e-01 -5.96133390e-02
+ -2.58632336e+00 -4.51557058e-01
+ -1.16570367e+00 -1.27065510e+00
+ 2.76187104e+00 2.21895451e-01
+ -3.80443767e+00 -1.66319902e+00
+ 9.84658633e-01 6.81475569e-01
+ 9.33814584e-01 -4.89335563e-02
+ -4.63427997e-01 1.72989539e-01
+ 1.82401546e+00 3.60164021e-01
+ -5.36521077e-01 -8.08691351e-01
+ -1.37367030e+00 -1.02126160e+00
+ -3.70310682e+00 -1.19840844e+00
+ -1.51894242e+00 -3.89510223e-01
+ -3.67347940e-01 -3.25540516e-02
+ -1.00988595e+00 1.82802194e-01
+ 2.01622795e+00 7.86367901e-01
+ 1.02440231e+00 8.79780360e-01
+ -3.05971480e+00 -8.40901527e-01
+ 2.73909457e+00 1.20558628e+00
+ 2.39559056e+00 1.10786694e+00
+ 1.65471544e+00 7.33824651e-01
+ 2.18546787e+00 6.41168955e-01
+ 1.47152266e+00 3.91839132e-01
+ 1.45811155e+00 5.21820495e-01
+ -4.27531469e-02 -3.52343068e-03
+ -9.54948010e-01 -1.52313876e-01
+ 7.57151215e-01 -5.68728854e-03
+ -8.46205751e-01 -7.54580229e-01
+ 4.14493548e+00 1.45532780e+00
+ 4.58688968e-01 -4.54012803e-02
+ -1.49295381e+00 -4.57471758e-01
+ 1.80020351e+00 8.13724973e-01
+ -5.82727738e+00 -2.18269581e+00
+ -2.09017809e+00 -1.18305177e+00
+ -2.31628303e+00 -7.21600235e-01
+ -8.09679091e-01 -1.49101752e-01
+ 8.88005605e-01 8.57940857e-01
+ -1.44148219e+00 -3.10926299e-01
+ 3.68828186e-01 -3.08848059e-01
+ -6.63267389e-01 -8.58950139e-02
+ -1.14702569e+00 -6.32147854e-01
+ -1.51741715e+00 -8.53330564e-01
+ -1.33903718e+00 -1.45875547e-01
+ 4.12485387e+00 1.85620435e+00
+ -2.42353639e+00 -2.92669850e-01
+ 1.88708583e+00 9.35984730e-01
+ 2.15585179e+00 6.30469051e-01
+ -1.13627973e-01 -1.62554045e-01
+ 2.04540494e+00 1.36599834e+00
+ 2.81591381e+00 1.60897941e+00
+ 3.02736260e-02 3.83255815e-03
+ 7.97634013e-02 -2.82035099e-01
+ -3.24607473e-01 -5.30065956e-01
+ -3.91862894e+00 -1.94083334e+00
+ 1.56360901e+00 7.93882743e-01
+ -1.03905772e+00 6.25590229e-01
+ 2.54746492e+00 1.64233560e+00
+ -4.80774423e-01 -8.92298032e-02
+ 9.06979990e-02 1.05020427e+00
+ -2.47521290e+00 -1.78275982e-01
+ -3.91871729e-01 3.80285423e-01
+ 1.00658382e+00 4.58947483e-01
+ 4.68102941e-01 1.02992741e+00
+ 4.44242568e-01 2.89870239e-01
+ 3.29684452e+00 1.44677474e+00
+ -2.24983007e+00 -9.65574499e-01
+ -3.54453926e-01 -3.99020325e-01
+ -3.87429665e+00 -1.90079739e+00
+ 2.02656674e+00 1.12444894e+00
+ 3.77011621e+00 1.43200852e+00
+ 1.61259275e+00 4.65417399e-01
+ 2.28725434e+00 6.79181395e-01
+ 2.75421009e+00 2.27327345e+00
+ -2.40894409e+00 -1.03926359e+00
+ 1.52996651e-01 -2.73373046e-02
+ -2.63218977e+00 -7.22802821e-01
+ 2.77688169e+00 1.15310186e+00
+ 1.18832341e+00 4.73457165e-01
+ -2.35536326e+00 -1.08034554e+00
+ -5.84221627e-01 1.03505984e-02
+ 2.96730300e+00 1.33478306e+00
+ -8.61947692e-01 6.09137051e-02
+ 8.22343921e-01 -8.14155286e-02
+ 1.75809015e+00 1.07921470e+00
+ 1.19501279e+00 1.05309972e+00
+ -1.75901792e+00 9.75320161e-02
+ 1.64398635e+00 9.54384323e-01
+ -2.21878052e-01 -3.64847144e-01
+ -2.03128968e+00 -8.57866419e-01
+ 1.86750633e+00 7.08524487e-01
+ 8.03972976e-01 3.47404314e-01
+ 3.41203749e+00 1.39810900e+00
+ 4.22397681e-01 -6.41440488e-01
+ -4.88493360e+00 -1.58967816e+00
+ -1.67649284e-01 -1.08485915e-01
+ 2.11489023e+00 1.50506158e+00
+ -1.81639929e+00 -3.85542192e-01
+ 2.24044819e-01 -1.45100577e-01
+ -3.39262411e+00 -1.44394324e+00
+ 1.68706599e+00 2.29199618e-01
+ -1.94093257e+00 -1.65975814e-01
+ 8.28143367e-01 5.92109281e-01
+ -8.29587998e-01 -9.57130831e-01
+ -1.50011401e+00 -8.36802092e-01
+ 2.40770449e+00 9.32820177e-01
+ 7.41391309e-02 3.12878473e-01
+ 1.87745264e-01 6.19231425e-01
+ 9.57622692e-01 -2.20640033e-01
+ 3.18479243e+00 1.02986233e+00
+ 2.43133846e+00 8.41302677e-01
+ -7.09963834e-01 1.99718943e-01
+ -2.88253498e-01 -3.62772094e-01
+ 5.14052574e+00 1.79304595e+00
+ -3.27930993e+00 -1.29177973e+00
+ -1.16723536e+00 1.29519656e-01
+ 1.04801056e+00 3.41508300e-01
+ -3.99256195e+00 -2.51176471e+00
+ -7.62824318e-01 -6.84242153e-01
+ 2.71524986e-02 5.35157164e-02
+ 3.26430102e+00 1.34887262e+00
+ -1.72357766e+00 -4.94524388e-01
+ -3.81149536e+00 -1.28121944e+00
+ 3.36919354e+00 1.10672075e+00
+ -3.14841757e+00 -7.10713767e-01
+ -3.16463676e+00 -7.58558435e-01
+ -2.44745969e+00 -1.08816514e+00
+ 2.79173264e-01 -2.19652051e-02
+ 4.15309883e-01 6.07502790e-01
+ -9.51007417e-01 -5.83976336e-01
+ -1.47929839e+00 -8.39850409e-01
+ 2.38335703e+00 6.16055149e-01
+ -7.47749031e-01 -5.56164928e-01
+ -3.65643622e-01 -5.06684411e-01
+ -1.76634163e+00 -7.86382097e-01
+ 6.76372222e-01 -3.06592181e-01
+ -1.33505058e+00 -1.18301441e-01
+ 3.59660179e+00 2.00424178e+00
+ -7.88912762e-02 8.71956146e-02
+ 1.22656397e+00 1.18149583e+00
+ 4.24919729e+00 1.20082355e+00
+ 2.94607456e+00 1.00676505e+00
+ 7.46061275e-02 4.41761753e-02
+ -2.47738025e-02 1.92737701e-01
+ -2.20509316e-01 -3.79163193e-01
+ -3.50222190e-01 3.58727299e-01
+ -3.64788014e+00 -1.36107312e+00
+ 3.56062799e+00 9.27032742e-01
+ 1.04317289e+00 6.08035970e-01
+ 4.06718718e-01 3.00628051e-01
+ 4.33158086e+00 2.25860714e+00
+ 2.13917145e-01 -1.72757967e-01
+ -1.40637998e+00 -1.14119465e+00
+ 3.61554872e+00 1.87797348e+00
+ 1.01726871e+00 5.70255097e-01
+ -7.04902551e-01 2.16444147e-01
+ -2.51492186e+00 -8.52997369e-01
+ 1.85097530e+00 1.15124496e+00
+ -8.67569714e-01 -3.05682432e-01
+ 8.07550858e-01 5.88901608e-01
+ 1.85186755e-01 -1.94589367e-01
+ -1.23378238e+00 -7.84128347e-01
+ -1.22713161e+00 -4.21218235e-01
+ 2.97751165e-01 2.81055275e-01
+ 4.77703554e+00 1.66265524e+00
+ 2.51549669e+00 7.49980674e-01
+ 2.76510822e-01 1.40456909e-01
+ 1.98740905e+00 -1.79608212e-01
+ 9.35429145e-01 8.44344180e-01
+ -1.20854492e+00 -5.00598453e-01
+ 2.29936219e+00 8.10236668e-01
+ 6.92555544e-01 -2.65891331e-01
+ -1.58050994e+00 2.31237821e-01
+ -1.50864880e+00 -9.49661690e-01
+ -1.27689206e+00 -7.18260016e-01
+ -3.12517127e+00 -1.75587113e+00
+ 8.16062912e-02 -6.56551804e-01
+ -5.02479939e-01 -4.67162543e-01
+ -5.47435788e+00 -2.47799576e+00
+ 1.95872901e-02 5.80874076e-01
+ -1.59064958e+00 -6.34554756e-01
+ -3.77521478e+00 -1.74301790e+00
+ 5.89628224e-01 8.55736553e-01
+ -1.81903543e+00 -7.50011008e-01
+ 1.38557775e+00 3.71490991e-01
+ 9.70032652e-01 -7.11356016e-01
+ 2.63539625e-01 -4.20994771e-01
+ 2.12154222e+00 8.19081400e-01
+ -6.56977937e-01 -1.37810098e-01
+ 8.91309581e-01 2.77864361e-01
+ -7.43693195e-01 -1.46293770e-01
+ 2.24447769e+00 4.00911438e-01
+ -2.25169262e-01 2.04148801e-02
+ 1.68744684e+00 9.47573007e-01
+ 2.73086373e-01 3.30877195e-01
+ 5.54294414e+00 2.14198009e+00
+ -8.49238733e-01 3.65603298e-02
+ 2.39685712e+00 1.17951039e+00
+ -2.58230528e+00 -5.52116673e-01
+ 2.79785277e+00 2.88833717e-01
+ -1.96576188e-01 1.11652123e+00
+ -4.69383301e-01 1.96496282e-01
+ -1.95011845e+00 -6.15235169e-01
+ 1.03379890e-02 2.33701239e-01
+ 4.18933607e-01 2.77939814e-01
+ -1.18473337e+00 -4.10051126e-01
+ -7.61499744e-01 -1.43658094e+00
+ -1.65586092e+00 -3.41615303e-01
+ -5.58523700e-02 -5.21837080e-01
+ -2.40331088e+00 -2.64521583e-01
+ 2.24925206e+00 6.79843335e-02
+ 1.46360479e+00 1.04271443e+00
+ -3.09255443e+00 -1.82548953e+00
+ 2.11325841e+00 1.14996627e+00
+ -8.70657797e-01 1.02461839e-01
+ -5.71056521e-01 9.71232588e-02
+ -3.37870752e+00 -1.54091877e+00
+ 1.03907189e+00 -1.35661392e-01
+ 8.40057486e-01 6.12172413e-02
+ -1.30998234e+00 -1.34077226e+00
+ 7.53744974e-01 1.49447350e-01
+ 9.13995056e-01 -1.81227962e-01
+ 2.28386229e-01 3.74498520e-01
+ 2.54829151e-01 -2.88802704e-01
+ 1.61709009e+00 2.09319193e-01
+ -1.12579380e+00 -5.95955338e-01
+ -2.69610726e+00 -2.76222736e-01
+ -2.63773329e+00 -7.84491970e-01
+ -2.62167427e+00 -1.54792874e+00
+ -4.80639856e-01 -1.30582102e-01
+ -1.26130891e+00 -8.86841840e-01
+ -1.24951950e+00 -1.18182622e+00
+ -1.40107574e+00 -9.13695575e-01
+ 4.99872179e-01 4.69014702e-01
+ -2.03550193e-02 -1.48859738e-01
+ -1.50189069e+00 -2.97714278e-02
+ -2.07846113e+00 -7.29937809e-01
+ -5.50576792e-01 -7.03151525e-01
+ -3.88069238e+00 -1.63215295e+00
+ 2.97032988e+00 6.43571144e-01
+ -1.85999273e-01 1.18107620e+00
+ 1.79249709e+00 6.65356160e-01
+ 2.68842472e+00 1.35703255e+00
+ 1.07675417e+00 1.39845588e-01
+ 8.01226349e-01 2.11392275e-01
+ 9.64329379e-01 3.96146195e-01
+ -8.22529511e-01 1.96080831e-01
+ 1.92481841e+00 4.62985744e-01
+ 3.69756927e-01 3.77135799e-01
+ 1.19807835e+00 8.87715050e-01
+ -1.01363587e+00 -2.48151636e-01
+ 8.53071010e-01 4.96887868e-01
+ -3.41120553e+00 -1.35401843e+00
+ -2.64787381e+00 -1.08690563e+00
+ -1.11416759e+00 -4.43848915e-01
+ 1.46242648e+00 6.17106076e-02
+ -7.52968881e-01 -9.20972209e-01
+ -1.22492228e+00 -5.40327617e-01
+ 1.08001827e+00 5.29593785e-01
+ -2.58706464e-01 1.13022085e-01
+ -4.27394011e-01 1.17864354e-02
+ -3.20728413e+00 -1.71224737e-01
+ 1.71398530e+00 8.68885893e-01
+ 2.12067866e+00 1.45092772e+00
+ 4.32782616e-01 -3.34117769e-01
+ 7.80084374e-01 -1.35100217e-01
+ -2.05547729e+00 -4.70217750e-01
+ 2.38379736e+00 1.09186058e+00
+ -2.80825477e+00 -1.03320187e+00
+ 2.63434576e+00 1.15671733e+00
+ -1.60936214e+00 1.91843035e-01
+ -5.02298769e+00 -2.32820708e+00
+ 1.90349195e+00 1.45215416e+00
+ 3.00232888e-01 3.24412586e-01
+ -2.46503943e+00 -1.19550010e+00
+ 1.06304233e+00 2.20136246e-01
+ -2.99101388e+00 -1.58299318e+00
+ 2.30071719e+00 1.12881362e+00
+ -2.37587247e+00 -8.08298336e-01
+ 7.27006308e-01 3.80828984e-01
+ 2.61199061e+00 1.56473491e+00
+ 8.33936357e-01 -1.42189425e-01
+ 3.13291605e+00 1.77771210e+00
+ 2.21917371e+00 5.68427075e-01
+ 2.38867649e+00 9.06637262e-01
+ -6.92959466e+00 -3.57682881e+00
+ 2.57904824e+00 5.93959108e-01
+ 2.71452670e+00 1.34436199e+00
+ 4.39988761e+00 2.13124672e+00
+ 5.71783077e-01 5.08346173e-01
+ -3.65399429e+00 -1.18192861e+00
+ 4.46176453e-01 3.75685594e-02
+ -2.97501495e+00 -1.69459236e+00
+ 1.60855728e+00 9.20930014e-01
+ -1.44270290e+00 -1.93922306e-01
+ 1.67624229e+00 1.66233866e+00
+ -1.42579598e+00 -1.44990145e-01
+ 1.19923176e+00 4.58490278e-01
+ -9.00068460e-01 5.09701825e-02
+ -1.69391694e+00 -7.60070300e-01
+ -1.36576440e+00 -5.24244256e-01
+ -1.03016748e+00 -3.44625878e-01
+ 2.40519313e+00 1.09947587e+00
+ 1.50365433e+00 1.06464802e+00
+ -1.07609727e+00 -3.68897187e-01
+ 2.44969069e+00 1.28486192e+00
+ -1.25610307e+00 -1.14644789e+00
+ 2.05962899e+00 4.31162369e-01
+ -7.15886908e-01 -6.11587804e-02
+ -6.92354119e-01 -7.85019920e-01
+ -1.63016508e+00 -5.96944975e-01
+ 1.90352536e+00 1.28197457e+00
+ -4.01535243e+00 -1.81934488e+00
+ -1.07534435e+00 -2.10544784e-01
+ 3.25500866e-01 7.69603661e-01
+ 2.18443365e+00 6.59773335e-01
+ 8.80856790e-01 6.39505913e-01
+ -2.23956372e-01 -4.65940132e-01
+ -1.06766519e+00 -5.38388505e-03
+ 7.25556863e-01 -2.91123488e-01
+ -4.69451411e-01 7.89182650e-02
+ 2.58146587e+00 1.29653243e+00
+ 1.53747468e-01 7.69239075e-01
+ -4.61152262e-01 -4.04151413e-01
+ 1.48183517e+00 8.10079506e-01
+ -1.83402614e+00 -1.36939322e+00
+ 1.49315501e+00 7.95225425e-01
+ 1.41922346e+00 1.05582774e-01
+ 1.57473493e-01 9.70795657e-01
+ -2.67603254e+00 -7.48562280e-01
+ -8.49156216e-01 -6.05762529e-03
+ 1.12944274e+00 3.67741591e-01
+ 1.94228071e-01 5.28188141e-01
+ -3.65610158e-01 4.05851838e-01
+ -1.98839111e+00 -1.38452764e+00
+ 2.73765752e+00 8.24150530e-01
+ 7.63728641e-01 3.51617707e-01
+ 5.78307267e+00 1.68103612e+00
+ 2.27547227e+00 3.60876164e-01
+ -3.50681697e+00 -1.74429984e+00
+ 4.01241184e+00 1.26227829e+00
+ 2.44946343e+00 9.06119057e-01
+ -2.96638941e+00 -9.01532322e-01
+ 1.11267643e+00 -3.43333381e-01
+ -6.61868994e-01 -3.44666391e-01
+ -8.34917179e-01 5.69478372e-01
+ -1.91888454e+00 -3.03791075e-01
+ 1.50397636e+00 8.31961240e-01
+ 6.12260198e+00 2.16851807e+00
+ 1.34093127e+00 8.86649385e-01
+ 1.48748519e+00 8.26273697e-01
+ 7.62243068e-01 2.64841396e-01
+ -2.17604986e+00 -3.54219958e-01
+ 2.64708640e-01 -4.38136718e-02
+ 1.44725372e+00 1.18499914e-01
+ -6.71259446e-01 -1.19526851e-01
+ 2.40134595e-01 -8.90042323e-02
+ -3.57238199e+00 -1.23166201e+00
+ -3.77626645e+00 -1.19533443e+00
+ -3.81101035e-01 -4.94160532e-01
+ -3.02758757e+00 -1.18436066e+00
+ 2.59116298e-01 1.38023047e+00
+ 4.17900116e+00 1.12065959e+00
+ 1.54598848e+00 2.89806755e-01
+ 1.00656475e+00 1.76974511e-01
+ -4.15730234e-01 -6.22681694e-01
+ -6.00903565e-01 -1.43256959e-01
+ -6.03652508e-01 -5.09936379e-01
+ -1.94096658e+00 -9.48789544e-01
+ -1.74464105e+00 -8.50491590e-01
+ 1.17652544e+00 1.88118317e+00
+ 2.35507776e+00 1.44000205e+00
+ 2.63067924e+00 1.06692988e+00
+ 2.88805386e+00 1.23924715e+00
+ 8.27595008e-01 5.75364692e-01
+ 3.91384216e-01 9.72781920e-02
+ -1.03866816e+00 -1.37567768e+00
+ -1.34777969e+00 -8.40266025e-02
+ -4.12904508e+00 -1.67618340e+00
+ 1.27918111e+00 3.52085961e-01
+ 4.15361174e-01 6.28896189e-01
+ -7.00539496e-01 4.80447955e-02
+ -1.62332639e+00 -5.98236485e-01
+ 1.45957300e+00 1.00305154e+00
+ -3.06875603e+00 -1.25897545e+00
+ -1.94708176e+00 4.85143006e-01
+ 3.55744156e+00 -1.07468822e+00
+ 1.21602223e+00 1.28768827e-01
+ 1.89093098e+00 -4.70835659e-01
+ -6.55759125e+00 2.70114082e+00
+ 8.96843535e-01 -3.98115252e-01
+ 4.13450429e+00 -2.32069236e+00
+ 2.37764218e+00 -1.09098890e+00
+ -1.11388901e+00 6.27083097e-01
+ -6.34116929e-01 4.62816387e-01
+ 2.90203079e+00 -1.33589143e+00
+ 3.17457598e+00 -5.13575945e-01
+ -1.76362299e+00 5.71820693e-01
+ 1.66103362e+00 -8.99466249e-01
+ -2.53947433e+00 8.40084780e-01
+ 4.36631397e-01 7.24234261e-02
+ -1.87589394e+00 5.08529113e-01
+ 4.49563965e+00 -9.43365992e-01
+ 1.78876299e+00 -1.27076149e+00
+ -1.16269107e-01 -4.55078316e-01
+ 1.92966079e+00 -8.05371385e-01
+ 2.20632583e+00 -9.00919345e-01
+ 1.52387824e+00 -4.82391996e-01
+ 8.04004564e-01 -2.73650595e-01
+ -7.75326067e-01 1.07469566e+00
+ 1.83226282e+00 -4.52173344e-01
+ 1.25079758e-01 -3.52895417e-02
+ -9.90957437e-01 8.55993130e-01
+ 1.71623322e+00 -7.08691667e-01
+ -2.86175924e+00 6.75160955e-01
+ -8.40817853e-01 -1.00361809e-01
+ 1.33393000e+00 -4.65788123e-01
+ 5.29394114e-01 -5.44881619e-02
+ -8.07435599e-01 8.27353370e-01
+ -4.33165824e+00 1.97299638e+00
+ 1.26452422e+00 -8.34070486e-01
+ 1.45996394e-02 2.97736043e-01
+ -1.64489287e+00 6.72839598e-01
+ -5.74234578e+00 3.20975117e+00
+ 2.13841341e-02 3.64514015e-01
+ 6.68084924e+00 -2.27464254e+00
+ -3.22881590e+00 8.01879324e-01
+ 3.02534313e-01 -4.56222796e-01
+ -5.84520734e+00 1.95678162e+00
+ 2.81515232e+00 -1.72101318e+00
+ -2.39620908e-01 2.69145522e-01
+ -7.41669691e-01 -2.30283281e-01
+ -2.15682714e+00 3.45313021e-01
+ 1.23475788e+00 -7.32276553e-01
+ -1.71816113e-01 1.20419560e-02
+ 1.89174235e+00 2.27435901e-01
+ -3.64511114e-01 1.72260361e-02
+ -3.24143860e+00 6.50125817e-01
+ -2.25707409e+00 5.66970751e-01
+ 1.03901456e+00 -1.00588433e+00
+ -5.09159710e+00 1.58736109e+00
+ 1.45534075e+00 -5.83787452e-01
+ 4.28879587e+00 -1.58006866e+00
+ 8.52384427e-01 -1.11042299e+00
+ 4.51431615e+00 -2.63844265e+00
+ -4.33042648e+00 1.86497078e+00
+ -2.13568046e+00 5.82559743e-01
+ -4.42568887e+00 1.26131214e+00
+ 3.15821315e+00 -1.61515905e+00
+ -3.14125204e+00 8.49604386e-01
+ 6.54152300e-01 -2.04624711e-01
+ -3.73374317e-01 9.94187820e-02
+ -3.96177282e+00 1.27245623e+00
+ 9.59825199e-01 -1.15547861e+00
+ 3.56902055e+00 -1.46591091e+00
+ 1.55433633e-02 6.93544345e-01
+ 1.15684646e+00 -4.99836352e-01
+ 3.11824573e+00 -4.75900506e-01
+ -8.61706369e-01 -3.50774059e-01
+ 9.89057391e-01 -7.16878802e-01
+ -4.94787870e+00 2.09137481e+00
+ 1.37777347e+00 -1.34946349e+00
+ -1.13161577e+00 8.05114754e-01
+ 8.12020675e-01 -1.04849421e+00
+ 4.73783881e+00 -2.26718812e+00
+ 8.99579366e-01 -8.89764451e-02
+ 4.78524868e+00 -2.25795843e+00
+ 1.75164590e+00 -1.73822209e-01
+ 1.30204590e+00 -7.26724717e-01
+ -7.26526403e-01 -5.23925361e-02
+ 2.01255351e+00 -1.69965366e+00
+ 9.87852740e-01 -4.63577220e-01
+ 2.45957762e+00 -1.29278962e+00
+ -3.13817948e+00 1.64433038e+00
+ -1.76302159e+00 9.62784302e-01
+ -1.91106331e+00 5.81460008e-01
+ -3.30883001e+00 1.30378978e+00
+ 5.54376450e-01 3.78814272e-01
+ 1.09982111e+00 -1.47969612e+00
+ -2.61300705e-02 -1.42573464e-01
+ -2.22096157e+00 7.75684440e-01
+ 1.70319323e+00 -2.89738444e-01
+ -1.43223842e+00 6.39284281e-01
+ 2.34360959e-01 -1.64379268e-01
+ -2.67147991e+00 9.46548086e-01
+ 1.51131425e+00 -4.91594395e-01
+ -2.48446856e+00 1.01286123e+00
+ 1.50534658e-01 -2.94620246e-01
+ -1.66966792e+00 1.67755508e+00
+ -1.50094241e+00 3.30163095e-01
+ 2.27681194e+00 -1.08064317e+00
+ 2.05122965e+00 -1.15165939e+00
+ -4.23509309e-01 -6.56906167e-02
+ 1.80084023e+00 -1.07228556e+00
+ -2.65769521e+00 1.18023206e+00
+ 2.02852676e+00 -8.06793574e-02
+ -4.49544185e+00 2.68200163e+00
+ -7.50043216e-01 1.17079331e+00
+ 6.80060893e-02 3.99055351e-01
+ -3.83634635e+00 1.38406887e+00
+ 3.24858545e-01 -9.25273218e-02
+ -2.19895100e+00 1.47819500e+00
+ -3.61569522e-01 -1.03188739e-01
+ 1.12180375e-01 -9.52696354e-02
+ -1.31477803e+00 1.79900570e-01
+ 2.39573628e+00 -6.09739269e-01
+ -1.00135700e+00 6.02837296e-01
+ -4.11994589e+00 2.49599192e+00
+ -1.54196236e-01 -4.84921951e-01
+ 5.92569908e-01 -1.87310359e-01
+ 3.85407741e+00 -1.50979925e+00
+ 5.17802528e+00 -2.26032607e+00
+ -1.37018916e+00 1.87111822e-01
+ 8.46682996e-01 -3.56676331e-01
+ -1.17559949e+00 5.29057734e-02
+ -5.56475671e-02 6.79049243e-02
+ 1.07851745e+00 -5.14535101e-01
+ -2.71622446e+00 1.00151846e+00
+ -1.08477208e+00 8.81391054e-01
+ 5.50755824e-01 -5.20577727e-02
+ 4.70885495e+00 -2.04220397e+00
+ -1.87375336e-01 -6.16962830e-02
+ 3.52097100e-01 2.21163550e-01
+ 7.07929984e-01 -1.75827590e-01
+ -1.22149219e+00 1.83084346e-01
+ 2.58247412e+00 -6.15914898e-01
+ -6.01206182e-01 -2.29832987e-01
+ 9.83360449e-01 -3.75870060e-01
+ -3.20027685e+00 1.35467480e+00
+ 1.79178978e+00 -1.38531981e+00
+ -3.30376867e-01 -1.16250192e-01
+ -1.89053055e+00 5.68463567e-01
+ -4.20604849e+00 1.65429681e+00
+ -1.01185529e+00 1.92801240e-01
+ -6.18819882e-01 5.42206996e-01
+ -5.08091672e+00 2.61598591e+00
+ -2.62570344e+00 2.51590658e+00
+ 3.05577906e+00 -1.49090609e+00
+ 2.77609677e+00 -1.37681378e+00
+ -7.93515301e-02 4.28072744e-01
+ -2.08359471e+00 8.94334295e-01
+ 2.20163801e+00 4.01127167e-02
+ -1.18145785e-01 -2.06822464e-01
+ -2.74788298e-01 2.96250607e-01
+ 1.59613555e+00 -3.87246203e-01
+ -3.82971472e-01 -3.39716093e-02
+ -4.20311307e-02 3.88529510e-01
+ 1.52128574e+00 -9.33138876e-01
+ -9.06584458e-01 -2.75016094e-02
+ 3.56216834e+00 -9.99384622e-01
+ 2.11964220e+00 -9.98749118e-02
+ 4.01203480e+00 -2.03032745e+00
+ -1.24171557e+00 1.97596725e-01
+ -1.57230455e+00 4.14126609e-01
+ -1.85484741e+00 5.40041563e-01
+ 1.76329831e+00 -6.95967734e-01
+ -2.29439232e-01 5.08669245e-01
+ -5.45124276e+00 2.26907549e+00
+ -5.71364288e-02 5.04476476e-01
+ 3.12468018e+00 -1.46358879e+00
+ 8.20017359e-01 6.51949028e-01
+ -1.33977500e+00 2.83634232e-04
+ -1.83311685e+00 1.23947117e+00
+ 6.31205922e-01 1.19792164e-02
+ -2.21967834e+00 6.94056232e-01
+ -1.41693842e+00 9.93526233e-01
+ -7.58885703e-01 6.78547347e-01
+ 3.60239086e+00 -1.08644935e+00
+ 6.72217073e-02 3.00036011e-02
+ -3.42680958e-01 -3.48049352e-01
+ 1.87546079e+00 -4.78018246e-01
+ 7.00485821e-01 -3.52905383e-01
+ -8.54580948e-01 8.17330861e-01
+ 8.19123706e-01 -5.73927281e-01
+ 2.70855639e-01 -3.08940052e-01
+ -1.05059952e+00 3.27873168e-01
+ 1.08282999e+00 4.84559349e-02
+ -7.89899220e-01 1.22291138e+00
+ -2.87939816e+00 7.17403497e-01
+ -2.08429452e+00 8.87409226e-01
+ 1.58409232e+00 -4.74123532e-01
+ 1.26882735e+00 1.59162510e-01
+ -2.53782993e+00 6.18253491e-01
+ -8.92757445e-01 3.35979011e-01
+ 1.31867900e+00 -1.17355054e+00
+ 1.14918879e-01 -5.35184038e-01
+ -1.70288738e-01 5.35868087e-02
+ 4.21355121e-01 5.41848690e-02
+ 2.07926943e+00 -5.72538144e-01
+ 4.08788970e-01 3.77655777e-01
+ -3.39631381e+00 9.84216764e-01
+ 2.94170163e+00 -1.83120916e+00
+ -7.94798752e-01 7.39889052e-01
+ 1.46555463e+00 -4.62275563e-01
+ 2.57255955e+00 -1.04671434e+00
+ 8.45042540e-01 -1.96952892e-01
+ -3.23526646e+00 1.60049846e+00
+ 3.21948565e+00 -8.88376674e-01
+ 1.43005104e+00 -9.21561086e-01
+ 8.82360506e-01 2.98403872e-01
+ -8.91168097e-01 1.01319072e+00
+ -5.13215241e-01 -2.47182649e-01
+ -1.35759444e+00 7.07450608e-02
+ -4.04550983e+00 2.23534867e+00
+ 1.39348883e+00 3.81637747e-01
+ -2.85676418e+00 1.53240862e+00
+ -1.37183120e+00 6.37977425e-02
+ -3.88195859e+00 1.73887145e+00
+ 1.19509776e+00 -6.25013512e-01
+ -2.80062734e+00 1.79840585e+00
+ 1.96558429e+00 -4.70997234e-01
+ 1.93111352e+00 -9.70318441e-01
+ 3.57991190e+00 -1.65065116e+00
+ 2.12831714e+00 -1.11531708e+00
+ -3.95661018e-01 -8.54339904e-02
+ -2.41630441e+00 1.65166304e+00
+ 7.55412624e-01 -1.53453579e-01
+ -1.77043450e+00 1.39928715e+00
+ -9.32631260e-01 8.73649199e-01
+ 1.53342205e+00 -8.39569765e-01
+ -6.29846924e-02 1.25023084e-01
+ 3.31509049e+00 -1.10733235e+00
+ -2.18957109e+00 3.07376993e-01
+ -2.35740747e+00 6.47437564e-01
+ -2.22142438e+00 8.47318938e-01
+ -6.51401147e-01 3.48398562e-01
+ 2.75763095e+00 -1.21390708e+00
+ 1.12550484e+00 -5.61412847e-01
+ -5.65053161e-01 6.74365205e-02
+ 1.68952456e+00 -6.57566096e-01
+ 8.95598401e-01 3.96738993e-01
+ -1.86537066e+00 9.44129208e-01
+ -2.59933294e+00 2.57423247e-01
+ -6.59598267e-01 1.91828851e-02
+ -2.64506676e+00 8.41783205e-01
+ -1.25911802e+00 5.52425066e-01
+ -1.39754507e+00 3.73689222e-01
+ 5.49550729e-02 1.35071215e+00
+ 3.31874811e+00 -1.05682424e+00
+ 3.63159604e+00 -1.42864695e+00
+ -4.45944617e+00 1.42889446e+00
+ 5.87314342e-01 -4.88892988e-01
+ -7.26130820e-01 1.51936106e-01
+ -1.79246441e+00 6.05888105e-01
+ -5.50948207e-01 6.21443081e-01
+ -3.17246063e-01 1.77213880e-01
+ -2.00098937e+00 1.23799074e+00
+ 4.33790961e+00 -1.08490465e+00
+ -2.03114114e+00 1.31613237e+00
+ -6.29216542e+00 1.92406317e+00
+ -1.60265624e+00 8.87947500e-01
+ 8.64465062e-01 -8.37416270e-01
+ -2.14273937e+00 8.05485900e-01
+ -2.36844256e+00 6.17915124e-01
+ -1.40429636e+00 6.78296866e-01
+ 9.99019988e-01 -5.84297572e-01
+ 7.38824546e-01 1.68838678e-01
+ 1.45681238e+00 3.04641461e-01
+ 2.15914949e+00 -3.43089227e-01
+ -1.23895930e+00 1.05339864e-01
+ -1.23162264e+00 6.46629863e-01
+ 2.28183862e+00 -9.24157063e-01
+ -4.29615882e-01 5.69130863e-01
+ -1.37449121e+00 -9.12032183e-01
+ -7.33890904e-01 -3.91865471e-02
+ 8.41400661e-01 -4.76002200e-01
+ -1.73349274e-01 -6.84143467e-02
+ 3.16042891e+00 -1.32651856e+00
+ -3.78244609e+00 2.38619718e+00
+ -3.69634380e+00 2.22368561e+00
+ 1.83766344e+00 -1.65675953e+00
+ -1.63206002e+00 1.19484469e+00
+ 3.68480064e-01 -5.70764494e-01
+ 3.61982479e-01 1.04274409e-01
+ 2.48863048e+00 -1.13285542e+00
+ -2.81896488e+00 9.47958768e-01
+ 5.74952901e-01 -2.75959392e-01
+ 3.72783275e-01 -3.48937848e-01
+ 1.95935716e+00 -1.06750415e+00
+ 5.19357531e+00 -2.32070803e+00
+ 4.09246149e+00 -1.89976700e+00
+ -3.36666087e-01 8.17645057e-02
+ 1.85453493e-01 3.76913151e-01
+ -3.06458262e+00 1.34106402e+00
+ -3.13796566e+00 7.00485099e-01
+ 1.42964058e+00 -1.35536932e-01
+ -1.23440423e-01 4.60094177e-02
+ -2.86753037e+00 -5.21724160e-02
+ 2.67113726e+00 -1.83746924e+00
+ -1.35335062e+00 1.28238073e+00
+ -2.43569899e+00 1.25998539e+00
+ 1.26036740e-01 -2.35416844e-01
+ -1.35725745e+00 7.37788491e-01
+ -3.80897538e-01 3.30757889e-01
+ 6.58694434e-01 -1.07566603e+00
+ 2.11273640e+00 -9.02260632e-01
+ 4.00755057e-01 -2.49229150e-02
+ -1.80095812e+00 9.73099742e-01
+ -2.68408372e+00 1.63737364e+00
+ -2.66079826e+00 7.47289412e-01
+ -9.92321439e-02 -1.49331396e-01
+ 4.45678251e+00 -1.80352394e+00
+ 1.35962915e+00 -1.31554389e+00
+ -7.76601417e-01 -9.66173523e-02
+ 1.68096348e+00 -6.27235133e-01
+ 1.53081227e-01 -3.54216830e-01
+ -1.54913095e+00 3.43689269e-01
+ 5.29187357e-02 -6.73916964e-01
+ -2.06606084e+00 8.34784242e-01
+ 1.73701179e+00 -6.06467340e-01
+ 1.55856757e+00 -2.58642780e-01
+ 1.04349101e+00 -4.43027348e-01
+ -1.02397719e+00 1.01308824e+00
+ -2.13860204e-01 -4.73347361e-01
+ -2.59004955e+00 1.43367853e+00
+ 7.98457679e-01 2.18621627e-02
+ -1.32974762e+00 4.61802208e-01
+ 3.21419359e-01 2.30723316e-02
+ 2.87201888e-02 6.24566672e-02
+ -1.22261418e+00 6.02340363e-01
+ 1.28750335e+00 -3.34839548e-02
+ -9.67952623e-01 4.34470505e-01
+ 2.02850324e+00 -9.05160255e-01
+ -4.13946010e+00 2.33779091e+00
+ -4.47508806e-01 3.06440495e-01
+ -3.91543394e+00 1.68251022e+00
+ -6.45193001e-01 5.29781162e-01
+ -2.15518916e-02 5.07278355e-01
+ -2.83356868e+00 1.00670227e+00
+ 1.82989749e+00 -1.37329222e+00
+ -1.09330213e+00 1.08560688e+00
+ 1.90533722e+00 -1.28905879e+00
+ 2.33986084e+00 2.30642626e-02
+ 8.01940220e-01 -1.63986962e+00
+ -4.23415165e+00 2.07530423e+00
+ 9.33382522e-01 -7.62917211e-01
+ -1.84033954e+00 1.07469401e+00
+ -2.81938669e+00 1.07342024e+00
+ -7.05169988e-01 2.13124943e-01
+ 5.09598137e-01 1.32725493e-01
+ -2.34558226e+00 8.62383168e-01
+ -1.70322072e+00 2.70893796e-01
+ 1.23652660e+00 -7.53216034e-02
+ 2.84660646e+00 -3.48178304e-02
+ 2.50250128e+00 -1.27770855e+00
+ -1.00279469e+00 8.77194218e-01
+ -4.34674121e-02 -2.12091350e-01
+ -5.84151289e-01 1.50382340e-01
+ -1.79024013e+00 4.24972808e-01
+ -1.23434666e+00 -8.85546570e-02
+ 1.36575412e+00 -6.42639880e-01
+ -1.98429947e+00 2.27650336e-01
+ 2.36253589e+00 -1.51340773e+00
+ 8.79157643e-01 6.84142159e-01
+ -2.18577755e+00 2.76526200e-01
+ -3.55473434e-01 8.29976561e-01
+ 1.16442595e+00 -5.97699411e-01
+ -7.35528097e-01 2.40318183e-01
+ -1.73702631e-01 7.33788663e-02
+ -1.40451745e+00 3.24899628e-01
+ -2.05434385e+00 5.68123738e-01
+ 8.47876642e-01 -5.74224294e-01
+ -6.91955602e-01 1.26009087e+00
+ 2.56574498e+00 -1.15602581e+00
+ 3.93306545e+00 -1.38398209e+00
+ -2.73230251e+00 4.89062581e-01
+ -1.04315474e+00 6.06335547e-01
+ 1.23231431e+00 -4.46675065e-01
+ -3.93035285e+00 1.43287651e+00
+ -1.02132111e+00 9.58919791e-01
+ -1.49425352e+00 1.06456165e+00
+ -6.26485337e-01 1.03791402e+00
+ -6.61772998e-01 2.63275425e-01
+ -1.80940386e+00 5.70767403e-01
+ 9.83720450e-01 -1.39449756e-01
+ -2.24619662e+00 9.01044870e-01
+ 8.94343014e-01 5.31038678e-02
+ 1.95518199e-01 -2.81343295e-01
+ -2.30533019e-01 -1.74478106e-01
+ -2.01550361e+00 5.55958010e-01
+ -4.36281469e+00 1.94374226e+00
+ -5.18530457e+00 2.89278357e+00
+ 2.67289101e+00 -2.98511449e-01
+ -1.53566179e+00 -1.00588944e-01
+ -6.09943217e-02 -1.56986047e-01
+ -5.22146452e+00 1.66209208e+00
+ -3.69777478e+00 2.26154873e+00
+ 2.24607181e-01 -4.86934960e-01
+ 2.49909450e+00 -1.03033370e+00
+ -1.07841120e+00 8.22388054e-01
+ -3.20697089e+00 1.09536143e+00
+ 3.43524232e+00 -1.47289362e+00
+ -5.65784134e-01 4.60365175e-01
+ -1.76714734e+00 1.57752346e-01
+ -7.77620365e-01 5.60153443e-01
+ 6.34399352e-01 -5.22339836e-01
+ 2.91011875e+00 -9.72623380e-01
+ -1.19286824e+00 6.32370253e-01
+ -2.18327609e-01 8.23953181e-01
+ 3.42430842e-01 1.37098055e-01
+ 1.28658034e+00 -9.11357320e-01
+ 2.06914465e+00 -6.67556382e-01
+ -6.69451020e-01 -6.38605102e-01
+ -2.09312398e+00 1.16743634e+00
+ -3.63778357e+00 1.91919157e+00
+ 8.74685911e-01 -1.09931208e+00
+ -3.91496791e+00 1.00808357e+00
+ 1.29621330e+00 -8.32239802e-01
+ 9.00222045e-01 -1.31159793e+00
+ -1.12242062e+00 1.98517079e-01
+ -3.71932852e-01 1.31667093e-01
+ -2.23829610e+00 1.26328346e+00
+ -2.08365062e+00 9.93385336e-01
+ -1.91082720e+00 7.45866855e-01
+ 4.38024917e+00 -2.05901118e+00
+ -2.28872886e+00 6.85279335e-01
+ 1.01274497e-01 -3.26227153e-01
+ -5.04447572e-01 -3.18619513e-01
+ 1.28537006e+00 -1.04573551e+00
+ -7.83175212e-01 1.54791645e-01
+ -3.89239175e+00 1.60017929e+00
+ -8.87877111e-01 -1.04968005e-01
+ 9.32215179e-01 -5.58691113e-01
+ -6.44977127e-01 -2.23018375e-01
+ 1.10141900e+00 -1.00666432e+00
+ 2.92755687e-01 -1.45480350e-01
+ 7.73580681e-01 -2.21150567e-01
+ -1.40873709e+00 7.61548044e-01
+ -8.89031805e-01 -3.48542923e-01
+ 4.16844267e-01 -2.39914494e-01
+ -4.64265832e-01 7.29581138e-01
+ 1.99835179e+00 -7.70542813e-01
+ 4.20523191e-02 -2.18783563e-01
+ -6.32611758e-01 -3.09926115e-01
+ 6.82912198e-02 -8.48327050e-01
+ 1.92425229e+00 -1.37876951e+00
+ 3.49461782e+00 -1.88354255e+00
+ -3.25209026e+00 1.49809395e+00
+ 6.59273182e-01 -2.37435654e-01
+ -1.15517300e+00 8.46134387e-01
+ 1.26756151e+00 -4.58988026e-01
+ -3.99178418e+00 2.04153008e+00
+ 7.05687841e-01 -6.83433306e-01
+ -1.61997342e+00 8.16577004e-01
+ -3.89750399e-01 4.29753250e-01
+ -2.53026432e-01 4.92861432e-01
+ -3.16788324e+00 4.44285524e-01
+ -7.86248901e-01 1.12753716e+00
+ -3.02351433e+00 1.28419015e+00
+ -1.30131355e+00 1.71226678e+00
+ -4.08843475e+00 1.62063214e+00
+ -3.09209403e+00 1.19958520e+00
+ 1.49102271e+00 -1.11834864e+00
+ -3.18059348e+00 5.74587042e-01
+ 2.06054867e+00 3.25797860e-03
+ -3.50999200e+00 2.02412428e+00
+ -8.26610023e-01 3.46528211e-01
+ 2.00546034e+00 -4.07333110e-01
+ -9.69941653e-01 4.80953753e-01
+ 4.47925660e+00 -2.33127314e+00
+ 2.03845790e+00 -9.90439915e-01
+ -1.11349191e+00 4.31183918e-01
+ -4.03628396e+00 1.68509679e+00
+ -1.48177601e+00 7.74322088e-01
+ 3.07369385e+00 -9.57465886e-01
+ 2.39011286e+00 -6.44506921e-01
+ 2.91561991e+00 -8.78627328e-01
+ 1.10212733e+00 -4.21637388e-01
+ 5.31985231e-01 -6.17445696e-01
+ -6.82340929e-01 -2.93529716e-01
+ 1.94290679e+00 -4.64268634e-01
+ 1.92262116e+00 -7.93142835e-01
+ 4.73762800e+00 -1.63654174e+00
+ -3.17848641e+00 8.05791391e-01
+ 4.08739432e+00 -1.80816807e+00
+ -7.60648826e-01 1.24216138e-01
+ -2.24716400e+00 7.90020937e-01
+ 1.64284052e+00 -7.18784070e-01
+ 1.04410012e-01 -7.11195880e-02
+ 2.18268225e+00 -7.01767831e-01
+ 2.06218013e+00 -8.70251746e-01
+ -1.35266581e+00 7.08456358e-01
+ -1.38157779e+00 5.14401086e-01
+ -3.28326008e+00 1.20988399e+00
+ 8.85358917e-01 -8.12213495e-01
+ -2.34067500e+00 3.67657353e-01
+ 3.96878127e+00 -1.66841450e+00
+ 1.36518053e+00 -8.33436812e-01
+ 5.25771988e-01 -5.06121987e-01
+ -2.25948361e+00 1.30663765e+00
+ -2.57662070e+00 6.32114628e-01
+ -3.43134685e+00 2.38106008e+00
+ 2.31571924e+00 -1.56566818e+00
+ -2.95397202e+00 1.05661888e+00
+ -1.35331242e+00 6.76383411e-01
+ 1.40977132e+00 -1.17775938e+00
+ 1.52561996e+00 -9.83147176e-01
+ 2.26550832e+00 -2.10464123e-02
+ 6.23371684e-01 -5.30768122e-01
+ -4.42356624e-01 9.72226986e-01
+ 2.31517901e+00 -1.08468105e+00
+ 1.97236640e+00 -1.42016619e+00
+ 3.18618687e+00 -1.45056343e+00
+ -2.75880360e+00 5.40254980e-01
+ -1.92916581e+00 1.45029864e-01
+ 1.90022524e+00 -6.03805754e-01
+ -1.05446211e+00 5.74361752e-01
+ 1.45990390e+00 -9.28233993e-01
+ 5.14960557e+00 -2.07564096e+00
+ -7.53104842e-01 1.55876958e-01
+ 8.09490983e-02 -8.58886384e-02
+ -1.56894969e+00 4.53497227e-01
+ 1.36944658e-01 5.60670875e-01
+ -5.32635329e-01 4.40309945e-01
+ 1.32507853e+00 -5.83670099e-01
+ 1.20676031e+00 -8.02296831e-01
+ -3.65023422e+00 1.17211368e+00
+ 1.53393850e+00 -6.17771312e-01
+ -3.99977129e+00 1.71415137e+00
+ 5.70705058e-01 -4.60771539e-01
+ -2.20608002e+00 1.07866596e+00
+ -1.09040244e+00 6.77441076e-01
+ -5.09886482e-01 -1.97282128e-01
+ -1.58062785e+00 6.18333697e-01
+ -1.53295020e+00 4.02168701e-01
+ -5.18580598e-01 2.25767177e-01
+ 1.59514316e+00 -2.54983617e-01
+ -5.91938655e+00 2.68223782e+00
+ 2.84200509e+00 -1.04685313e+00
+ 1.31298664e+00 -1.16672614e+00
+ -2.36660033e+00 1.81359460e+00
+ 6.94163290e-02 3.76658816e-01
+ 2.33973934e+00 -8.33173023e-01
+ -8.24640389e-01 7.83717285e-01
+ -1.02888281e+00 1.04680766e+00
+ 1.34750745e+00 -5.89568160e-01
+ -2.48761231e+00 7.44199284e-01
+ -1.04501559e+00 4.72326911e-01
+ -3.14610089e+00 1.89843692e+00
+ 2.13003416e-01 5.76633620e-01
+ -1.69239608e+00 5.66070021e-01
+ 1.80491280e+00 -9.31701080e-01
+ -6.94362572e-02 6.96026587e-01
+ 1.36502578e+00 -6.85599000e-02
+ -7.76764337e-01 3.64328661e-01
+ -2.67322167e+00 6.80150021e-01
+ 1.84338485e+00 -1.18487494e+00
+ 2.88009231e+00 -1.25700411e+00
+ 1.17114433e+00 -7.69727080e-01
+ 2.11576167e+00 2.81502116e-01
+ -1.51470088e+00 2.61553540e-01
+ 1.18923669e-01 -1.17890202e-01
+ 4.48359786e+00 -1.81427466e+00
+ -1.27055948e+00 9.92388998e-01
+ -8.00276606e-01 9.11326621e-02
+ 7.51764024e-01 -1.03676498e-01
+ 1.35769348e-01 -2.11470084e-01
+ 2.50731332e+00 -1.12418270e+00
+ -2.49752781e-01 7.81224033e-02
+ -6.23037902e-01 3.16599691e-01
+ -3.93772902e+00 1.37195391e+00
+ 1.74256361e+00 -1.12363582e+00
+ -1.49737281e+00 5.98828310e-01
+ 7.75592115e-01 -4.64733802e-01
+ -2.26027693e+00 1.36991118e+00
+ -1.62849836e+00 7.36899107e-01
+ 2.36850751e+00 -9.32126872e-01
+ 5.86169745e+00 -2.49342512e+00
+ -5.37092226e-01 1.23821274e+00
+ 2.80535867e+00 -1.93363302e+00
+ -1.77638106e+00 9.10050276e-01
+ 3.02692018e+00 -1.60774676e+00
+ 1.97833084e+00 -1.50636531e+00
+ 9.09168906e-01 -8.83799359e-01
+ 2.39769655e+00 -7.56977869e-01
+ 1.47283981e+00 -1.06749890e+00
+ 2.92060943e-01 -6.07040605e-01
+ -2.09278201e+00 7.71858590e-01
+ 7.10015905e-01 -5.42768432e-01
+ -2.16826169e-01 1.56897896e-01
+ 4.56288247e+00 -2.08912680e+00
+ -6.63374020e-01 6.67325183e-01
+ 1.80564442e+00 -9.76366134e-01
+ 3.28720168e+00 -4.66575145e-01
+ -1.60463695e-01 -2.58428153e-01
+ 1.78590750e+00 -3.96427146e-01
+ 2.75950306e+00 -1.82102856e+00
+ -1.18234310e+00 6.28073320e-01
+ 4.11415835e+00 -2.33551216e+00
+ 1.38721004e+00 -2.77450622e-01
+ -2.94903545e+00 1.74813352e+00
+ 8.67290400e-01 -6.51667894e-01
+ 2.70022274e+00 -8.11832480e-01
+ -2.06766146e+00 8.24047249e-01
+ 3.90717142e+00 -1.20155758e+00
+ -2.95102809e+00 1.36667968e+00
+ 6.08815147e+00 -2.60737974e+00
+ 2.78576476e+00 -7.86628755e-01
+ -3.26258407e+00 1.09302450e+00
+ 1.59849422e+00 -1.09705202e+00
+ -2.50600710e-01 1.63243175e-01
+ -4.90477087e-01 -4.57729572e-01
+ -1.24837181e+00 3.22157840e-01
+ -2.46341049e+00 1.06517849e+00
+ 9.62880751e-01 4.56962496e-01
+ 3.99964487e-01 2.07472802e-01
+ 6.36657705e-01 -3.46400942e-02
+ 4.91231407e-02 -1.40289235e-02
+ -4.66683524e-02 -3.72326100e-01
+ -5.22049702e-01 -1.70440260e-01
+ 5.27062938e-01 -2.32628395e-01
+ -2.69440318e+00 1.18914874e+00
+ 3.65087539e+00 -1.53427267e+00
+ -1.16546364e-01 4.93245392e-02
+ 7.55931384e-01 -3.02980139e-01
+ 2.06338745e+00 -6.24841225e-01
+ 1.31177908e-01 7.29338183e-01
+ 1.48021784e+00 -6.39509896e-01
+ -5.98656707e-01 2.84525503e-01
+ -2.18611080e+00 1.79549812e+00
+ -2.91673624e+00 2.15772237e-01
+ -8.95591350e-01 7.68250538e-01
+ 1.36139762e+00 -1.93845144e-01
+ 5.45730414e+00 -2.28114404e+00
+ 3.22747247e-01 9.33582332e-01
+ -1.46384504e+00 1.12801186e-01
+ 4.26728166e-01 -2.33481242e-01
+ -1.41327270e+00 8.16103740e-01
+ -2.53998067e-01 1.44906646e-01
+ -1.32436467e+00 1.87556361e-01
+ -3.77313086e+00 1.32896038e+00
+ 3.77651731e+00 -1.76548043e+00
+ -2.45297093e+00 1.32571926e+00
+ -6.55900588e-01 3.56921462e-01
+ 9.25558722e-01 -4.51988954e-01
+ 1.20732231e+00 -3.02821614e-01
+ 3.72660154e-01 -1.89365208e-01
+ -1.77090939e+00 9.18087975e-01
+ 3.01127567e-01 2.67965829e-01
+ -1.76708900e+00 4.62069259e-01
+ -2.71812099e+00 1.57233508e+00
+ -5.35297633e-01 4.99231535e-01
+ 1.50507631e+00 -9.85763646e-01
+ 3.00424787e+00 -1.29837562e+00
+ -4.99311105e-01 3.91086482e-01
+ 1.30125207e+00 -1.26247924e-01
+ 4.01699483e-01 -4.46909391e-01
+ -1.33635257e+00 5.12068703e-01
+ 1.39229757e+00 -9.10974858e-01
+ -1.74229508e+00 1.49475978e+00
+ -1.21489414e+00 4.04193753e-01
+ -3.36537605e-01 -6.74335427e-01
+ -2.79186828e-01 8.48314720e-01
+ -2.03080140e+00 1.66599815e+00
+ -3.53064281e-01 -7.68582906e-04
+ -5.30305657e+00 2.91091546e+00
+ -1.20049972e+00 8.26578358e-01
+ 2.95906989e-01 2.40215920e-01
+ -1.42955534e+00 4.63480310e-01
+ -1.87856619e+00 8.21459385e-01
+ -2.71124720e+00 1.80246843e+00
+ -3.06933780e+00 1.22235760e+00
+ 5.21935582e-01 -1.27298218e+00
+ -1.34175797e+00 7.69018937e-01
+ -1.81962785e+00 1.15528991e+00
+ -3.99227550e-01 2.93821598e-01
+ 1.22533179e+00 -4.73846323e-01
+ -2.08068359e-01 -1.75039817e-01
+ -2.03068526e+00 1.50370503e+00
+ -3.27606113e+00 1.74906330e+00
+ -4.37802587e-01 -2.26956048e-01
+ -7.69774213e-02 -3.54922468e-01
+ 6.47160749e-02 -2.07334721e-01
+ -1.37791524e+00 4.43766709e-01
+ 3.29846803e+00 -1.04060799e+00
+ -3.63704046e+00 1.05800226e+00
+ -1.26716116e+00 1.13077353e+00
+ 1.98549075e+00 -1.31864807e+00
+ 1.85159500e+00 -5.78629560e-01
+ -1.55295206e+00 1.23655857e+00
+ 6.76026255e-01 9.18824125e-02
+ 1.23418960e+00 -4.68162027e-01
+ 2.43186642e+00 -9.22422440e-01
+ -3.18729701e+00 1.77582673e+00
+ -4.02945613e+00 1.14303496e+00
+ -1.92694576e-01 1.03301431e-01
+ 1.89554730e+00 -4.60128096e-01
+ -2.55626581e+00 1.16057084e+00
+ 6.89144365e-01 -9.94982900e-01
+ -4.44680606e+00 2.19751983e+00
+ -3.15196193e+00 1.18762993e+00
+ -1.17434977e+00 1.04534656e+00
+ 8.58386984e-02 -1.03947487e+00
+ 3.33354973e-01 5.54813610e-01
+ -9.37631808e-01 3.33450150e-01
+ -2.50232471e+00 5.39720635e-01
+ 1.03611949e+00 -7.16304095e-02
+ -2.05556816e-02 -3.28992265e-01
+ -2.24176201e+00 1.13077506e+00
+ 4.53583688e+00 -1.10710212e+00
+ 4.77389762e-01 -8.99445512e-01
+ -2.69075551e+00 6.83176866e-01
+ -2.21779724e+00 1.16916849e+00
+ -1.09669056e+00 2.10044765e-01
+ -8.45367920e-01 -8.45951423e-02
+ 4.37558941e-01 -6.95904256e-01
+ 1.84884195e+00 -1.71205136e-01
+ -8.36371957e-01 5.62862478e-01
+ 1.27786531e+00 -1.33362147e+00
+ 2.90684492e+00 -7.49892184e-01
+ -3.38652716e+00 1.51180670e+00
+ -1.30945978e+00 7.09261928e-01
+ -7.50471924e-01 -5.24637889e-01
+ 1.18580718e+00 -9.97943971e-04
+ -7.55395645e+00 3.19273590e+00
+ 1.72822535e+00 -1.20996962e+00
+ 5.67374320e-01 6.19573416e-01
+ -2.99163781e+00 1.79721534e+00
+ 1.49862187e+00 -6.05631846e-02
+ 1.79503506e+00 -4.90419706e-01
+ 3.85626054e+00 -1.95396324e+00
+ -9.39188410e-01 7.96498057e-01
+ 2.91986664e+00 -1.29392724e+00
+ -1.54265750e+00 6.40727933e-01
+ 1.14919794e+00 1.20834257e-01
+ 2.00936817e+00 -1.53728359e+00
+ 3.72468420e+00 -1.38704612e+00
+ -1.27794802e+00 3.48543179e-01
+ 3.63294077e-01 5.70623314e-01
+ 1.49381016e+00 -6.04500534e-01
+ 2.98912256e+00 -1.72295726e+00
+ -1.80833817e+00 2.94907625e-01
+ -3.19669622e+00 1.31888700e+00
+ 1.45889401e+00 -8.88448639e-01
+ -2.80045388e+00 1.01207060e+00
+ -4.78379567e+00 1.48646520e+00
+ 2.25510003e+00 -7.13372461e-01
+ -9.74441433e-02 -2.17766373e-01
+ 2.64468496e-01 -3.60842698e-01
+ -5.98821713e+00 3.20197892e+00
+ 2.67030213e-01 -5.36386416e-01
+ 2.24546960e+00 -8.13464649e-01
+ -4.89171414e-01 3.86255031e-01
+ -7.45713706e-01 6.29800380e-01
+ -3.30460503e-01 3.85127284e-01
+ -4.19588147e+00 1.52793198e+00
+ 5.42078582e-01 -2.61642741e-02
+ 4.24938513e-01 -5.72936751e-01
+ 2.82717288e+00 -6.75355024e-01
+ -1.44741788e+00 5.03578028e-01
+ -1.65547573e+00 7.76444277e-01
+ 2.20361170e+00 -1.40835680e+00
+ -3.69540235e+00 2.32953767e+00
+ -1.41909357e-01 2.28989778e-01
+ 1.92838879e+00 -8.72525737e-01
+ 1.40708100e+00 -6.81849638e-02
+ 1.24988112e+00 -1.39470590e-01
+ -2.39435855e+00 7.26587655e-01
+ 7.03985028e-01 4.85403277e-02
+ 4.05214529e+00 -9.16928318e-01
+ 3.74198837e-01 -5.04192358e-01
+ -8.43374127e-01 2.36064018e-01
+ -3.32253349e-01 7.47840055e-01
+ -6.03725210e+00 1.95173337e+00
+ 4.60829865e+00 -1.51191309e+00
+ -1.46247098e+00 1.11140916e+00
+ -9.60111157e-01 -1.23189114e-01
+ -7.49613187e-01 4.53614129e-01
+ -5.77838219e-01 2.07366469e-02
+ 8.07652950e-01 -5.16272662e-01
+ -6.02556049e-01 5.05318649e-01
+ -1.28712445e-01 2.57836512e-01
+ -5.27662820e+00 2.11790737e+00
+ 5.40819308e+00 -2.15366022e+00
+ 9.37742513e-02 -1.60221751e-01
+ 4.55902865e+00 -1.24646307e+00
+ -9.06582589e-01 1.92928110e-01
+ 2.99928996e+00 -8.04301218e-01
+ -3.24317381e+00 1.80076061e+00
+ 3.20421743e-01 8.76524679e-01
+ -5.29606705e-01 -3.16717696e-01
+ -1.77264560e+00 7.52686776e-01
+ -1.51706824e+00 8.43755103e-01
+ 1.52759111e+00 -7.86814243e-01
+ 4.74845617e-01 4.21319700e-01
+ 6.97829149e-01 -8.15664881e-01
+ 3.09564973e+00 -1.06202469e+00
+ 2.95320379e+00 -1.98963943e+00
+ -4.23033224e+00 1.41013338e+00
+ 1.48576206e+00 8.02908511e-02
+ 4.52041627e+00 -2.04620399e+00
+ 6.58403922e-01 -7.60781799e-01
+ 2.10667543e-01 1.15241731e-01
+ 1.77702583e+00 -8.10271859e-01
+ 2.41277385e+00 -1.46972042e+00
+ 1.50685525e+00 -1.99272545e-01
+ 7.61665522e-01 -4.11276152e-01
+ 1.18352312e+00 -9.59908608e-01
+ -3.32031305e-01 8.07500132e-02
+ 1.16813118e+00 -1.73095194e-01
+ 1.18363346e+00 -5.41565052e-01
+ 5.17702179e-01 -7.62442035e-01
+ 4.57401006e-01 -1.45951115e-02
+ 1.49377115e-01 2.99571605e-01
+ 1.40399453e+00 -1.30160353e+00
+ 5.26231567e-01 3.52783752e-01
+ -1.91136514e+00 4.24228635e-01
+ 1.74156701e+00 -9.92076776e-01
+ -4.89323391e+00 2.32483507e+00
+ 2.54011209e+00 -8.80366295e-01
+ -5.56925706e-01 1.48842026e-01
+ -2.35904668e+00 9.60474853e-01
+ 1.42216971e+00 -4.67062761e-01
+ -1.10809680e+00 7.68684300e-01
+ 4.09674726e+00 -1.90795680e+00
+ -2.23048923e+00 9.03812542e-01
+ 6.57025763e-01 1.36514871e-01
+ 2.10944145e+00 -9.78897838e-02
+ 1.22552525e+00 -2.50303867e-01
+ 2.84620103e-01 -5.30164020e-01
+ -2.13562585e+00 1.03503056e+00
+ 1.32414902e-01 -8.14190240e-03
+ -5.82433561e-01 3.21020292e-01
+ -5.06473247e-01 3.11530419e-01
+ 1.57162465e+00 -1.20763919e+00
+ -1.43155284e+00 -2.51203698e-02
+ -1.47093713e+00 -1.39620999e-01
+ -2.65765643e+00 1.06091403e+00
+ 2.45992927e+00 -5.88815836e-01
+ -1.28440162e+00 -1.99377398e-01
+ 6.11257504e-01 -3.73577401e-01
+ -3.46606103e-01 6.06081290e-01
+ 3.76687505e+00 -8.80181424e-01
+ -1.03725103e+00 1.45177517e+00
+ 2.76659936e+00 -1.09361320e+00
+ -3.61311296e+00 9.75032455e-01
+ 3.22878655e+00 -9.69497365e-01
+ 1.43560379e+00 -5.52524585e-01
+ 2.94042153e+00 -1.79747037e+00
+ 1.30739580e+00 2.47989248e-01
+ -4.05056982e-01 1.22831715e+00
+ -2.25827421e+00 2.30604626e-01
+ 3.69262926e-01 4.32714650e-02
+ -5.52064063e-01 6.07806340e-01
+ 7.03325987e+00 -2.17956730e+00
+ -2.37823835e-01 -8.28068639e-01
+ -4.84279888e-01 5.67765194e-01
+ -3.15863410e+00 1.02241617e+00
+ -3.39561593e+00 1.36876374e+00
+ -2.78482934e+00 6.81641104e-01
+ -4.37604334e+00 2.23826340e+00
+ -2.54049692e+00 8.22676745e-01
+ 3.73264822e+00 -9.93498732e-01
+ -3.49536064e+00 1.84771519e+00
+ 9.81801604e-01 -5.21278776e-01
+ 1.52996831e+00 -1.27386206e+00
+ -9.23490293e-01 5.29099482e-01
+ -2.76999461e+00 9.24831872e-01
+ -3.30029834e-01 -2.49645555e-01
+ -1.71156166e+00 5.44940854e-01
+ -2.37009487e+00 5.83826982e-01
+ -3.03216865e+00 1.04922722e+00
+ -2.19539936e+00 1.37558730e+00
+ 1.15350207e+00 -6.15318535e-01
+ 4.62011792e+00 -2.46714517e+00
+ 1.52627952e-02 -1.00618283e-01
+ -1.10399342e+00 4.87413533e-01
+ 3.55448194e+00 -9.10394190e-01
+ -5.21890321e+00 2.44710745e+00
+ 1.54289749e+00 -6.54269311e-01
+ 2.67935674e+00 -9.92758863e-01
+ 1.05801310e+00 2.60054285e-02
+ 1.52509097e+00 -4.08768600e-01
+ 3.27576917e+00 -1.28769406e+00
+ 1.71008412e-01 -2.68739994e-01
+ -9.83351344e-04 7.02495897e-02
+ -7.60795056e-03 1.61968285e-01
+ -1.80620472e+00 4.24934471e-01
+ 2.32023297e-02 -2.57284559e-01
+ 3.98219478e-01 -4.65361935e-01
+ 6.63476988e-01 -3.29823196e-02
+ 4.00154707e+00 -1.01792211e+00
+ -1.50286870e+00 9.46875359e-01
+ -2.22717585e+00 7.50636195e-01
+ -3.47381508e-01 -6.51596975e-01
+ 2.08076453e+00 -8.22800165e-01
+ 2.05099963e+00 -4.00868250e-01
+ 3.52576988e-02 -2.54418565e-01
+ 1.57342042e+00 -7.62166492e-02
+ -1.47019722e+00 3.40861172e-01
+ -1.21156090e+00 3.21891246e-01
+ 3.79729047e+00 -1.54350764e+00
+ 1.26459678e-02 6.99203693e-01
+ 1.53974177e-01 4.68643204e-01
+ -1.73923561e-01 -1.26229768e-01
+ 4.54644993e+00 -2.13951783e+00
+ 1.46022547e-01 -4.57084165e-01
+ 6.50048037e+00 -2.78872609e+00
+ -1.51934912e+00 1.03216768e+00
+ -3.06483575e+00 1.81101446e+00
+ -2.38212125e+00 9.19559042e-01
+ -1.81319611e+00 8.10545112e-01
+ 1.70951294e+00 -6.10712680e-01
+ 1.67974156e+00 -1.51241453e+00
+ -5.94795113e+00 2.56893813e+00
+ 3.62633110e-01 -7.46965304e-01
+ -2.44042594e+00 8.52761797e-01
+ 3.32412550e+00 -1.28439899e+00
+ 4.74860766e+00 -1.72821964e+00
+ 1.29072541e+00 -8.24872902e-01
+ -1.69450702e+00 4.09600876e-01
+ 1.29705411e+00 1.22300809e-01
+ -2.63597613e+00 8.55612913e-01
+ 9.28467301e-01 -2.63550114e-02
+ 2.44670264e+00 -4.10123002e-01
+ 1.06408206e+00 -5.03361942e-01
+ 5.12384049e-02 -1.27116595e-02
+ -1.06731272e+00 -1.76205029e-01
+ -9.45454582e-01 3.74404917e-01
+ 2.54343689e+00 -7.13810545e-01
+ -2.54460335e+00 1.31590265e+00
+ 1.89864233e+00 -3.98436339e-01
+ -1.93990133e+00 6.01474630e-01
+ -1.35938824e+00 4.00751788e-01
+ 2.38567018e+00 -6.13904880e-01
+ 2.18748050e-01 2.62631712e-01
+ -2.01388788e+00 1.41474031e+00
+ 2.74014581e+00 -1.27448105e+00
+ -2.13828583e+00 1.13616144e+00
+ 5.98730932e+00 -2.53430080e+00
+ -1.72872795e+00 1.53702057e+00
+ -2.53263962e+00 1.27342410e+00
+ 1.34326968e+00 -1.99395088e-01
+ 3.83352666e-01 -1.25683065e-01
+ -2.35630657e+00 5.54116983e-01
+ -1.94900838e+00 5.76270178e-01
+ -1.36699108e+00 -3.40904824e-01
+ -2.34727346e+00 -1.93054940e-02
+ -3.82779777e+00 1.83025664e+00
+ -4.31602080e+00 9.21605705e-01
+ 5.54098133e-01 2.33991419e-01
+ -4.53591188e+00 1.99833353e+00
+ -3.92715909e+00 1.83231482e+00
+ 3.91344440e-01 -1.11355111e-01
+ 3.48576363e+00 -1.41379449e+00
+ -1.42858690e+00 3.84532286e-01
+ 1.79519859e+00 -9.23486448e-01
+ 8.49691242e-01 -1.76551084e-01
+ 1.53618138e+00 8.23835015e-02
+ 5.91476520e-02 3.88296940e-02
+ 1.44837346e+00 -7.24097604e-01
+ -6.79008418e-01 4.04078097e-01
+ 2.87555510e+00 -9.51825076e-01
+ -1.12379101e+00 2.93457714e-01
+ 1.45263980e+00 -6.01960544e-01
+ -2.55741621e-01 9.26233518e-01
+ 3.54570714e+00 -1.41521877e+00
+ -1.61542388e+00 6.57844512e-01
+ -3.22844269e-01 3.02823546e-01
+ 1.03523913e+00 -6.92730711e-01
+ 1.11084909e+00 -3.50823642e-01
+ 3.41268693e+00 -1.90865862e+00
+ 7.67062858e-01 -9.48792160e-01
+ -5.49798016e+00 1.71139960e+00
+ 1.14865798e+00 -6.12669150e-01
+ -2.18256680e+00 7.78634462e-01
+ 4.78857389e+00 -2.55555085e+00
+ -1.85555569e+00 8.04311615e-01
+ -4.22278799e+00 2.01162524e+00
+ -1.56556149e+00 1.54353907e+00
+ -3.11527864e+00 1.65973526e+00
+ 2.66342611e+00 -1.20449402e+00
+ 1.57635314e+00 -1.48716308e-01
+ -6.35606865e-01 2.59701180e-01
+ 1.02431976e+00 -6.76929904e-01
+ 1.12973772e+00 1.49473892e-02
+ -9.12758116e-01 2.21533933e-01
+ -2.98014470e+00 1.71651189e+00
+ 2.74016965e+00 -9.47893923e-01
+ -3.47830591e+00 1.34941430e+00
+ 1.74757562e+00 -3.72503752e-01
+ 5.55820383e-01 -6.47992466e-01
+ -1.19871928e+00 9.82429151e-01
+ -2.53040133e+00 2.10671307e+00
+ -1.94085605e+00 1.38938137e+00
diff --git a/data/mllib/sample_fpgrowth.txt b/data/mllib/sample_fpgrowth.txt
new file mode 100644
index 0000000000000..c451583e51317
--- /dev/null
+++ b/data/mllib/sample_fpgrowth.txt
@@ -0,0 +1,6 @@
+r z h k p
+z y x w v u t s
+s x o n r
+x z y m t s q e
+z
+x z y r q t p
diff --git a/data/mllib/sample_isotonic_regression_data.txt b/data/mllib/sample_isotonic_regression_data.txt
new file mode 100644
index 0000000000000..d257b509d4d37
--- /dev/null
+++ b/data/mllib/sample_isotonic_regression_data.txt
@@ -0,0 +1,100 @@
+0.24579296,0.01
+0.28505864,0.02
+0.31208567,0.03
+0.35900051,0.04
+0.35747068,0.05
+0.16675166,0.06
+0.17491076,0.07
+0.04181540,0.08
+0.04793473,0.09
+0.03926568,0.10
+0.12952575,0.11
+0.00000000,0.12
+0.01376849,0.13
+0.13105558,0.14
+0.08873024,0.15
+0.12595614,0.16
+0.15247323,0.17
+0.25956145,0.18
+0.20040796,0.19
+0.19581846,0.20
+0.15757267,0.21
+0.13717491,0.22
+0.19020908,0.23
+0.19581846,0.24
+0.20091790,0.25
+0.16879143,0.26
+0.18510964,0.27
+0.20040796,0.28
+0.29576747,0.29
+0.43396226,0.30
+0.53391127,0.31
+0.52116267,0.32
+0.48546660,0.33
+0.49209587,0.34
+0.54156043,0.35
+0.59765426,0.36
+0.56144824,0.37
+0.58592555,0.38
+0.52983172,0.39
+0.50178480,0.40
+0.52626211,0.41
+0.58286588,0.42
+0.64660887,0.43
+0.68077511,0.44
+0.74298827,0.45
+0.64864865,0.46
+0.67261601,0.47
+0.65782764,0.48
+0.69811321,0.49
+0.63029067,0.50
+0.61601224,0.51
+0.63233044,0.52
+0.65323814,0.53
+0.65323814,0.54
+0.67363590,0.55
+0.67006629,0.56
+0.51555329,0.57
+0.50892402,0.58
+0.33299337,0.59
+0.36206017,0.60
+0.43090260,0.61
+0.45996940,0.62
+0.56348802,0.63
+0.54920959,0.64
+0.48393677,0.65
+0.48495665,0.66
+0.46965834,0.67
+0.45181030,0.68
+0.45843957,0.69
+0.47118817,0.70
+0.51555329,0.71
+0.58031617,0.72
+0.55481897,0.73
+0.56297807,0.74
+0.56603774,0.75
+0.57929628,0.76
+0.64762876,0.77
+0.66241713,0.78
+0.69301377,0.79
+0.65119837,0.80
+0.68332483,0.81
+0.66598674,0.82
+0.73890872,0.83
+0.73992861,0.84
+0.84242733,0.85
+0.91330954,0.86
+0.88016318,0.87
+0.90719021,0.88
+0.93115757,0.89
+0.93115757,0.90
+0.91942886,0.91
+0.92911780,0.92
+0.95665477,0.93
+0.95002550,0.94
+0.96940337,0.95
+1.00000000,0.96
+0.89801122,0.97
+0.90311066,0.98
+0.90362060,0.99
+0.83477817,1.0
\ No newline at end of file
diff --git a/data/mllib/sample_lda_data.txt b/data/mllib/sample_lda_data.txt
new file mode 100644
index 0000000000000..2e76702ca9d67
--- /dev/null
+++ b/data/mllib/sample_lda_data.txt
@@ -0,0 +1,12 @@
+1 2 6 0 2 3 1 1 0 0 3
+1 3 0 1 3 0 0 2 0 0 1
+1 4 1 0 0 4 9 0 1 2 0
+2 1 0 3 0 0 5 0 2 3 9
+3 1 1 9 3 0 2 0 0 1 3
+4 2 0 3 4 5 1 1 1 4 0
+2 1 0 3 0 0 5 0 2 2 9
+1 1 1 9 2 1 2 0 0 1 3
+4 4 0 3 4 2 1 3 0 0 0
+2 8 2 0 3 0 2 0 2 7 2
+1 1 1 9 0 2 2 0 0 3 3
+4 1 0 0 4 5 1 3 0 1 0
diff --git a/dev/change-version-to-2.10.sh b/dev/change-version-to-2.10.sh
index 7473c20d28e09..15e0c73b4295e 100755
--- a/dev/change-version-to-2.10.sh
+++ b/dev/change-version-to-2.10.sh
@@ -16,5 +16,11 @@
# See the License for the specific language governing permissions and
# limitations under the License.
#
+
+# Note that this will not necessarily work as intended with non-GNU sed (e.g. OS X)
+
find . -name 'pom.xml' | grep -v target \
- | xargs -I {} sed -i -e 's|\(artifactId.*\)_2.11|\1_2.10|g' {}
+ | xargs -I {} sed -i -e 's/\(artifactId.*\)_2.11/\1_2.10/g' {}
+
+# Also update in parent POM
+sed -i -e '0,/2.112.10' pom.xml
diff --git a/dev/change-version-to-2.11.sh b/dev/change-version-to-2.11.sh
index 3957a9f3ba258..c0a8cb4f825c7 100755
--- a/dev/change-version-to-2.11.sh
+++ b/dev/change-version-to-2.11.sh
@@ -17,5 +17,10 @@
# limitations under the License.
#
+# Note that this will not necessarily work as intended with non-GNU sed (e.g. OS X)
+
find . -name 'pom.xml' | grep -v target \
- | xargs -I {} sed -i -e 's|\(artifactId.*\)_2.10|\1_2.11|g' {}
+ | xargs -I {} sed -i -e 's/\(artifactId.*\)_2.10/\1_2.11/g' {}
+
+# Also update in parent POM
+sed -i -e '0,/2.102.11' pom.xml
diff --git a/dev/check-license b/dev/check-license
index a006f65710d6d..39943f882b6ca 100755
--- a/dev/check-license
+++ b/dev/check-license
@@ -31,7 +31,7 @@ acquire_rat_jar () {
printf "Attempting to fetch rat\n"
JAR_DL="${JAR}.part"
if [ $(command -v curl) ]; then
- curl --silent "${URL}" > "$JAR_DL" && mv "$JAR_DL" "$JAR"
+ curl -L --silent "${URL}" > "$JAR_DL" && mv "$JAR_DL" "$JAR"
elif [ $(command -v wget) ]; then
wget --quiet ${URL} -O "$JAR_DL" && mv "$JAR_DL" "$JAR"
else
diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh
index b2a7e092a0291..b5a67dd783b93 100755
--- a/dev/create-release/create-release.sh
+++ b/dev/create-release/create-release.sh
@@ -22,8 +22,9 @@
# Expects to be run in a totally empty directory.
#
# Options:
-# --package-only only packages an existing release candidate
-#
+# --skip-create-release Assume the desired release tag already exists
+# --skip-publish Do not publish to Maven central
+# --skip-package Do not package and upload binary artifacts
# Would be nice to add:
# - Send output to stderr and have useful logging in stdout
@@ -33,6 +34,9 @@ ASF_PASSWORD=${ASF_PASSWORD:-XXX}
GPG_PASSPHRASE=${GPG_PASSPHRASE:-XXX}
GIT_BRANCH=${GIT_BRANCH:-branch-1.0}
RELEASE_VERSION=${RELEASE_VERSION:-1.2.0}
+# Allows publishing under a different version identifier than
+# was present in the actual release sources (e.g. rc-X)
+PUBLISH_VERSION=${PUBLISH_VERSION:-$RELEASE_VERSION}
NEXT_VERSION=${NEXT_VERSION:-1.2.1}
RC_NAME=${RC_NAME:-rc2}
@@ -51,7 +55,7 @@ set -e
GIT_TAG=v$RELEASE_VERSION-$RC_NAME
-if [[ ! "$@" =~ --package-only ]]; then
+if [[ ! "$@" =~ --skip-create-release ]]; then
echo "Creating release commit and publishing to Apache repository"
# Artifact publishing
git clone https://$ASF_USERNAME:$ASF_PASSWORD@git-wip-us.apache.org/repos/asf/spark.git \
@@ -87,12 +91,25 @@ if [[ ! "$@" =~ --package-only ]]; then
git commit -a -m "Preparing development version $next_ver"
git push origin $GIT_TAG
git push origin HEAD:$GIT_BRANCH
- git checkout -f $GIT_TAG
+ popd
+ rm -rf spark
+fi
+
+if [[ ! "$@" =~ --skip-publish ]]; then
+ git clone https://$ASF_USERNAME:$ASF_PASSWORD@git-wip-us.apache.org/repos/asf/spark.git
+ pushd spark
+ git checkout --force $GIT_TAG
+
+ # Substitute in case published version is different than released
+ old="^\( \{2,4\}\)${RELEASE_VERSION}<\/version>$"
+ new="\1${PUBLISH_VERSION}<\/version>"
+ find . -name pom.xml | grep -v dev | xargs -I {} sed -i \
+ -e "s/${old}/${new}/" {}
# Using Nexus API documented here:
# https://support.sonatype.com/entries/39720203-Uploading-to-a-Staging-Repository-via-REST-API
echo "Creating Nexus staging repository"
- repo_request="Apache Spark $GIT_TAG "
+ repo_request="Apache Spark $GIT_TAG (published as $PUBLISH_VERSION) "
out=$(curl -X POST -d "$repo_request" -u $ASF_USERNAME:$ASF_PASSWORD \
-H "Content-Type:application/xml" -v \
$NEXUS_ROOT/profiles/$NEXUS_PROFILE/start)
@@ -101,13 +118,13 @@ if [[ ! "$@" =~ --package-only ]]; then
rm -rf $SPARK_REPO
- mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \
+ build/mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \
-Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \
clean install
./dev/change-version-to-2.11.sh
-
- mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \
+
+ build/mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \
-Dscala-2.11 -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \
clean install
@@ -144,7 +161,7 @@ if [[ ! "$@" =~ --package-only ]]; then
done
echo "Closing nexus staging repository"
- repo_request="$staged_repo_id Apache Spark $GIT_TAG "
+ repo_request="$staged_repo_id Apache Spark $GIT_TAG (published as $PUBLISH_VERSION) "
out=$(curl -X POST -d "$repo_request" -u $ASF_USERNAME:$ASF_PASSWORD \
-H "Content-Type:application/xml" -v \
$NEXUS_ROOT/profiles/$NEXUS_PROFILE/finish)
@@ -155,88 +172,96 @@ if [[ ! "$@" =~ --package-only ]]; then
rm -rf spark
fi
-# Source and binary tarballs
-echo "Packaging release tarballs"
-git clone https://git-wip-us.apache.org/repos/asf/spark.git
-cd spark
-git checkout --force $GIT_TAG
-release_hash=`git rev-parse HEAD`
-
-rm .gitignore
-rm -rf .git
-cd ..
-
-cp -r spark spark-$RELEASE_VERSION
-tar cvzf spark-$RELEASE_VERSION.tgz spark-$RELEASE_VERSION
-echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour --output spark-$RELEASE_VERSION.tgz.asc \
- --detach-sig spark-$RELEASE_VERSION.tgz
-echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md MD5 spark-$RELEASE_VERSION.tgz > \
- spark-$RELEASE_VERSION.tgz.md5
-echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md SHA512 spark-$RELEASE_VERSION.tgz > \
- spark-$RELEASE_VERSION.tgz.sha
-rm -rf spark-$RELEASE_VERSION
-
-make_binary_release() {
- NAME=$1
- FLAGS=$2
- cp -r spark spark-$RELEASE_VERSION-bin-$NAME
-
- cd spark-$RELEASE_VERSION-bin-$NAME
-
- # TODO There should probably be a flag to make-distribution to allow 2.11 support
- if [[ $FLAGS == *scala-2.11* ]]; then
- ./dev/change-version-to-2.11.sh
- fi
-
- ./make-distribution.sh --name $NAME --tgz $FLAGS 2>&1 | tee ../binary-release-$NAME.log
+if [[ ! "$@" =~ --skip-package ]]; then
+ # Source and binary tarballs
+ echo "Packaging release tarballs"
+ git clone https://git-wip-us.apache.org/repos/asf/spark.git
+ cd spark
+ git checkout --force $GIT_TAG
+ release_hash=`git rev-parse HEAD`
+
+ rm .gitignore
+ rm -rf .git
cd ..
- cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz .
- rm -rf spark-$RELEASE_VERSION-bin-$NAME
-
- echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \
- --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \
- --detach-sig spark-$RELEASE_VERSION-bin-$NAME.tgz
- echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \
- MD5 spark-$RELEASE_VERSION-bin-$NAME.tgz > \
- spark-$RELEASE_VERSION-bin-$NAME.tgz.md5
- echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \
- SHA512 spark-$RELEASE_VERSION-bin-$NAME.tgz > \
- spark-$RELEASE_VERSION-bin-$NAME.tgz.sha
-}
-
-
-make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" &
-make_binary_release "hadoop1-scala2.11" "-Phive -Dscala-2.11" &
-make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" &
-make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" &
-make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" &
-make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" &
-make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" &
-make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" &
-wait
-
-# Copy data
-echo "Copying release tarballs"
-rc_folder=spark-$RELEASE_VERSION-$RC_NAME
-ssh $ASF_USERNAME@people.apache.org \
- mkdir /home/$ASF_USERNAME/public_html/$rc_folder
-scp spark-* \
- $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_folder/
-
-# Docs
-cd spark
-build/sbt clean
-cd docs
-# Compile docs with Java 7 to use nicer format
-JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build
-echo "Copying release documentation"
-rc_docs_folder=${rc_folder}-docs
-ssh $ASF_USERNAME@people.apache.org \
- mkdir /home/$ASF_USERNAME/public_html/$rc_docs_folder
-rsync -r _site/* $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_docs_folder
-
-echo "Release $RELEASE_VERSION completed:"
-echo "Git tag:\t $GIT_TAG"
-echo "Release commit:\t $release_hash"
-echo "Binary location:\t http://people.apache.org/~$ASF_USERNAME/$rc_folder"
-echo "Doc location:\t http://people.apache.org/~$ASF_USERNAME/$rc_docs_folder"
+
+ cp -r spark spark-$RELEASE_VERSION
+ tar cvzf spark-$RELEASE_VERSION.tgz spark-$RELEASE_VERSION
+ echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour --output spark-$RELEASE_VERSION.tgz.asc \
+ --detach-sig spark-$RELEASE_VERSION.tgz
+ echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md MD5 spark-$RELEASE_VERSION.tgz > \
+ spark-$RELEASE_VERSION.tgz.md5
+ echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md SHA512 spark-$RELEASE_VERSION.tgz > \
+ spark-$RELEASE_VERSION.tgz.sha
+ rm -rf spark-$RELEASE_VERSION
+
+ # Updated for each binary build
+ make_binary_release() {
+ NAME=$1
+ FLAGS=$2
+ ZINC_PORT=$3
+ cp -r spark spark-$RELEASE_VERSION-bin-$NAME
+
+ cd spark-$RELEASE_VERSION-bin-$NAME
+
+ # TODO There should probably be a flag to make-distribution to allow 2.11 support
+ if [[ $FLAGS == *scala-2.11* ]]; then
+ ./dev/change-version-to-2.11.sh
+ fi
+
+ export ZINC_PORT=$ZINC_PORT
+ echo "Creating distribution: $NAME ($FLAGS)"
+ ./make-distribution.sh --name $NAME --tgz $FLAGS -DzincPort=$ZINC_PORT 2>&1 > \
+ ../binary-release-$NAME.log
+ cd ..
+ cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz .
+
+ echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \
+ --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \
+ --detach-sig spark-$RELEASE_VERSION-bin-$NAME.tgz
+ echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \
+ MD5 spark-$RELEASE_VERSION-bin-$NAME.tgz > \
+ spark-$RELEASE_VERSION-bin-$NAME.tgz.md5
+ echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \
+ SHA512 spark-$RELEASE_VERSION-bin-$NAME.tgz > \
+ spark-$RELEASE_VERSION-bin-$NAME.tgz.sha
+ }
+
+ # We increment the Zinc port each time to avoid OOM's and other craziness if multiple builds
+ # share the same Zinc server.
+ make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" "3030" &
+ make_binary_release "hadoop1-scala2.11" "-Phive -Dscala-2.11" "3031" &
+ make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" "3032" &
+ make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" "3033" &
+ make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" "3034" &
+ make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" "3035" &
+ make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" "3036" &
+ make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" "3037" &
+ wait
+ rm -rf spark-$RELEASE_VERSION-bin-*/
+
+ # Copy data
+ echo "Copying release tarballs"
+ rc_folder=spark-$RELEASE_VERSION-$RC_NAME
+ ssh $ASF_USERNAME@people.apache.org \
+ mkdir /home/$ASF_USERNAME/public_html/$rc_folder
+ scp spark-* \
+ $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_folder/
+
+ # Docs
+ cd spark
+ sbt/sbt clean
+ cd docs
+ # Compile docs with Java 7 to use nicer format
+ JAVA_HOME="$JAVA_7_HOME" PRODUCTION=1 RELEASE_VERSION="$RELEASE_VERSION" jekyll build
+ echo "Copying release documentation"
+ rc_docs_folder=${rc_folder}-docs
+ ssh $ASF_USERNAME@people.apache.org \
+ mkdir /home/$ASF_USERNAME/public_html/$rc_docs_folder
+ rsync -r _site/* $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_docs_folder
+
+ echo "Release $RELEASE_VERSION completed:"
+ echo "Git tag:\t $GIT_TAG"
+ echo "Release commit:\t $release_hash"
+ echo "Binary location:\t http://people.apache.org/~$ASF_USERNAME/$rc_folder"
+ echo "Doc location:\t http://people.apache.org/~$ASF_USERNAME/$rc_docs_folder"
+fi
diff --git a/dev/create-release/known_translations b/dev/create-release/known_translations
index b74e4ee8a330b..0a599b5a65549 100644
--- a/dev/create-release/known_translations
+++ b/dev/create-release/known_translations
@@ -57,3 +57,37 @@ watermen - Yadong Qi
witgo - Guoqiang Li
xinyunh - Xinyun Huang
zsxwing - Shixiong Zhu
+Bilna - Bilna P
+DoingDone9 - Doing Done
+Earne - Ernest
+FlytxtRnD - Meethu Mathew
+GenTang - Gen TANG
+JoshRosen - Josh Rosen
+MechCoder - Manoj Kumar
+OopsOutOfMemory - Sheng Li
+Peishen-Jia - Peishen Jia
+SaintBacchus - Huang Zhaowei
+azagrebin - Andrey Zagrebin
+bzz - Alexander Bezzubov
+fjiang6 - Fan Jiang
+gasparms - Gaspar Munoz
+guowei2 - Guo Wei
+hhbyyh - Yuhao Yang
+hseagle - Peng Xu
+javadba - Stephen Boesch
+jbencook - Ben Cook
+kul - Kuldeep
+ligangty - Gang Li
+marsishandsome - Liangliang Gu
+medale - Markus Dale
+nemccarthy - Nathan McCarthy
+nxwhite-str - Nate Crosswhite
+seayi - Xiaohua Yi
+tianyi - Yi Tian
+uncleGen - Uncle Gen
+viper-kun - Xu Kun
+x1- - Yuri Saito
+zapletal-martin - Martin Zapletal
+zuxqoj - Shekhar Bansal
+mingyukim - Mingyu Kim
+sigmoidanalytics - Mayur Rustagi
diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py
index dfa924d2aa0ba..3062e9c3c6651 100755
--- a/dev/merge_spark_pr.py
+++ b/dev/merge_spark_pr.py
@@ -244,6 +244,8 @@ def resolve_jira_issue(merge_branches, comment, default_jira_id=""):
versions = asf_jira.project_versions("SPARK")
versions = sorted(versions, key=lambda x: x.name, reverse=True)
versions = filter(lambda x: x.raw['released'] is False, versions)
+ # Consider only x.y.z versions
+ versions = filter(lambda x: re.match('\d+\.\d+\.\d+', x.name), versions)
default_fix_versions = map(lambda x: fix_version_from_branch(x, versions).name, merge_branches)
for v in default_fix_versions:
diff --git a/dev/run-tests b/dev/run-tests
index 2257a566bb1bb..d6935a61c6d29 100755
--- a/dev/run-tests
+++ b/dev/run-tests
@@ -36,7 +36,7 @@ function handle_error () {
}
-# Build against the right verison of Hadoop.
+# Build against the right version of Hadoop.
{
if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then
if [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop1.0" ]; then
@@ -77,7 +77,7 @@ export SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Pkinesis-asl"
fi
}
-# Only run Hive tests if there are sql changes.
+# Only run Hive tests if there are SQL changes.
# Partial solution for SPARK-1455.
if [ -n "$AMPLAB_JENKINS" ]; then
git fetch origin master:master
@@ -141,29 +141,41 @@ echo "========================================================================="
CURRENT_BLOCK=$BLOCK_BUILD
{
+ HIVE_BUILD_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver"
+ HIVE_12_BUILD_ARGS="$HIVE_BUILD_ARGS -Phive-0.12.0"
- # NOTE: echo "q" is needed because sbt on encountering a build file with failure
- # (either resolution or compilation) prompts the user for input either q, r, etc
- # to quit or retry. This echo is there to make it not block.
- # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a
- # single argument!
- # QUESTION: Why doesn't 'yes "q"' work?
- # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work?
# First build with Hive 0.12.0 to ensure patches do not break the Hive 0.12.0 build
- HIVE_12_BUILD_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver -Phive-0.12.0"
echo "[info] Compile with Hive 0.12.0"
- echo -e "q\n" \
- | build/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \
- | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including"
+ [ -d "lib_managed" ] && rm -rf lib_managed
+ echo "[info] Building Spark with these arguments: $HIVE_12_BUILD_ARGS"
+
+ if [ "${AMPLAB_JENKINS_BUILD_TOOL}" == "maven" ]; then
+ build/mvn $HIVE_12_BUILD_ARGS clean package -DskipTests
+ else
+ # NOTE: echo "q" is needed because sbt on encountering a build file with failure
+ # (either resolution or compilation) prompts the user for input either q, r, etc
+ # to quit or retry. This echo is there to make it not block.
+ # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a
+ # single argument!
+ # QUESTION: Why doesn't 'yes "q"' work?
+ # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work?
+ echo -e "q\n" \
+ | build/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \
+ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including"
+ fi
# Then build with default Hive version (0.13.1) because tests are based on this version
echo "[info] Compile with Hive 0.13.1"
- rm -rf lib_managed
- echo "[info] Building Spark with these arguments: $SBT_MAVEN_PROFILES_ARGS"\
- " -Phive -Phive-thriftserver"
- echo -e "q\n" \
- | build/sbt $SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver package assembly/assembly \
- | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including"
+ [ -d "lib_managed" ] && rm -rf lib_managed
+ echo "[info] Building Spark with these arguments: $HIVE_BUILD_ARGS"
+
+ if [ "${AMPLAB_JENKINS_BUILD_TOOL}" == "maven" ]; then
+ build/mvn $HIVE_BUILD_ARGS clean package -DskipTests
+ else
+ echo -e "q\n" \
+ | build/sbt $HIVE_BUILD_ARGS package assembly/assembly \
+ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including"
+ fi
}
echo ""
@@ -183,24 +195,28 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS
if [ -n "$_SQL_TESTS_ONLY" ]; then
# This must be an array of individual arguments. Otherwise, having one long string
# will be interpreted as a single test, which doesn't work.
- SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "mllib/test")
+ SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "hive-thriftserver/test" "mllib/test")
else
SBT_MAVEN_TEST_ARGS=("test")
fi
echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS ${SBT_MAVEN_TEST_ARGS[@]}"
- # NOTE: echo "q" is needed because sbt on encountering a build file with failure
- # (either resolution or compilation) prompts the user for input either q, r, etc
- # to quit or retry. This echo is there to make it not block.
- # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a
- # single argument!
- # "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array.
- # QUESTION: Why doesn't 'yes "q"' work?
- # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work?
- echo -e "q\n" \
- | build/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \
- | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including"
+ if [ "${AMPLAB_JENKINS_BUILD_TOOL}" == "maven" ]; then
+ build/mvn test $SBT_MAVEN_PROFILES_ARGS --fail-at-end
+ else
+ # NOTE: echo "q" is needed because sbt on encountering a build file with failure
+ # (either resolution or compilation) prompts the user for input either q, r, etc
+ # to quit or retry. This echo is there to make it not block.
+ # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a
+ # single argument!
+ # "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array.
+ # QUESTION: Why doesn't 'yes "q"' work?
+ # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work?
+ echo -e "q\n" \
+ | build/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \
+ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including"
+ fi
}
echo ""
diff --git a/docs/_config.yml b/docs/_config.yml
index e2db274e1f619..0652927a8ce9b 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -10,6 +10,7 @@ kramdown:
include:
- _static
+ - _modules
# These allow the documentation to be updated with newer releases
# of Spark, Scala, and Mesos.
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html
index 8841f7675d35e..2e88b3093652d 100755
--- a/docs/_layouts/global.html
+++ b/docs/_layouts/global.html
@@ -7,7 +7,9 @@
{{ page.title }} - Spark {{site.SPARK_VERSION_SHORT}} Documentation
-
+ {% if page.description %}
+
+ {% endif %}
{% if page.redirect %}
@@ -69,7 +71,7 @@
- Spark Programming Guide
- Spark Streaming
- - Spark SQL
+ - DataFrames and SQL
- MLlib (Machine Learning)
- GraphX (Graph Processing)
- Bagel (Pregel on Spark)
diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md
index 7e55131754a3f..c2fe6b0e286ce 100644
--- a/docs/bagel-programming-guide.md
+++ b/docs/bagel-programming-guide.md
@@ -1,6 +1,7 @@
---
layout: global
-title: Bagel Programming Guide
+displayTitle: Bagel Programming Guide
+title: Bagel
---
**Bagel will soon be superseded by [GraphX](graphx-programming-guide.html); we recommend that new users try GraphX instead.**
diff --git a/docs/building-spark.md b/docs/building-spark.md
index fb93017861ed0..57d0ca834f460 100644
--- a/docs/building-spark.md
+++ b/docs/building-spark.md
@@ -9,6 +9,10 @@ redirect_from: "building-with-maven.html"
Building Spark using Maven requires Maven 3.0.4 or newer and Java 6+.
+**Note:** Building Spark with Java 7 or later can create JAR files that may not be
+readable with early versions of Java 6, due to the large number of files in the JAR
+archive. Build with Java 6 if this is an issue for your deployment.
+
# Building with `build/mvn`
Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows:
@@ -111,9 +115,9 @@ To produce a Spark package compiled with Scala 2.11, use the `-Dscala-2.11` prop
dev/change-version-to-2.11.sh
mvn -Pyarn -Phadoop-2.4 -Dscala-2.11 -DskipTests clean package
-Scala 2.11 support in Spark is experimental and does not support a few features.
-Specifically, Spark's external Kafka library and JDBC component are not yet
-supported in Scala 2.11 builds.
+Scala 2.11 support in Spark does not support a few features due to dependencies
+which are themselves not Scala 2.11 ready. Specifically, Spark's external
+Kafka library and JDBC component are not yet supported in Scala 2.11 builds.
# Spark Tests in Maven
@@ -137,15 +141,18 @@ We use the scala-maven-plugin which supports incremental and continuous compilat
should run continuous compilation (i.e. wait for changes). However, this has not been tested
extensively. A couple of gotchas to note:
+
* it only scans the paths `src/main` and `src/test` (see
[docs](http://scala-tools.org/mvnsites/maven-scala-plugin/usage_cc.html)), so it will only work
from within certain submodules that have that structure.
+
* you'll typically need to run `mvn install` from the project root for compilation within
specific submodules to work; this is because submodules that depend on other submodules do so via
the `spark-parent` module).
Thus, the full flow for running continuous-compilation of the `core` submodule may look more like:
- ```
+
+```
$ mvn install
$ cd core
$ mvn scala:cc
@@ -156,14 +163,6 @@ Thus, the full flow for running continuous-compilation of the `core` submodule m
For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troubleshooting, refer to the
[wiki page for IDE setup](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-IDESetup).
-# Building Spark Debian Packages
-
-The Maven build includes support for building a Debian package containing the assembly 'fat-jar', PySpark, and the necessary scripts and configuration files. This can be created by specifying the following:
-
- mvn -Pdeb -DskipTests clean package
-
-The debian package can then be found under assembly/target. We added the short commit hash to the file name so that we can distinguish individual packages built for SNAPSHOT versions.
-
# Running Java 8 Test Suites
Running only Java 8 tests and nothing else.
diff --git a/docs/configuration.md b/docs/configuration.md
index e4e4b8d516b75..63fc99e7d3e29 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -1,6 +1,7 @@
---
layout: global
-title: Spark Configuration
+displayTitle: Spark Configuration
+title: Configuration
---
* This will become a table of contents (this text will be scraped).
{:toc}
@@ -69,7 +70,9 @@ each line consists of a key and a value separated by whitespace. For example:
Any values specified as flags or in the properties file will be passed on to the application
and merged with those specified through SparkConf. Properties set directly on the SparkConf
take highest precedence, then flags passed to `spark-submit` or `spark-shell`, then options
-in the `spark-defaults.conf` file.
+in the `spark-defaults.conf` file. A few configuration keys have been renamed since earlier
+versions of Spark; in such cases, the older key names are still accepted, but take lower
+precedence than any instance of the newer key.
## Viewing Spark Properties
@@ -93,14 +96,6 @@ of the most common options to set are:
The name of your application. This will appear in the UI and in log data.
-
- spark.master
- (none)
-
- The cluster manager to connect to. See the list of
- allowed master URL's.
-
-
spark.driver.cores
1
@@ -108,23 +103,6 @@ of the most common options to set are:
Number of cores to use for the driver process, only in cluster mode.
-
- spark.driver.memory
- 512m
-
- Amount of memory to use for the driver process, i.e. where SparkContext is initialized.
- (e.g. 512m
, 2g
).
-
-
-
- spark.executor.memory
- 512m
-
- Amount of memory to use per executor process, in the same format as JVM memory strings
- (e.g. 512m
, 2g
).
-
-
-
spark.driver.maxResultSize
1g
@@ -137,38 +115,35 @@ of the most common options to set are:
- spark.serializer
- org.apache.spark.serializer.
JavaSerializer
+ spark.driver.memory
+ 512m
- Class to use for serializing objects that will be sent over the network or need to be cached
- in serialized form. The default of Java serialization works with any Serializable Java object
- but is quite slow, so we recommend using
- org.apache.spark.serializer.KryoSerializer
and configuring Kryo serialization
- when speed is necessary. Can be any subclass of
-
- org.apache.spark.Serializer
.
+ Amount of memory to use for the driver process, i.e. where SparkContext is initialized.
+ (e.g. 512m
, 2g
).
+
+
Note: In client mode, this config must not be set through the SparkConf
+ directly in your application, because the driver JVM has already started at that point.
+ Instead, please set this through the --driver-memory
command line option
+ or in your default properties file.
- spark.kryo.classesToRegister
- (none)
+ spark.executor.memory
+ 512m
- If you use Kryo serialization, give a comma-separated list of custom class names to register
- with Kryo.
- See the tuning guide for more details.
+ Amount of memory to use per executor process, in the same format as JVM memory strings
+ (e.g. 512m
, 2g
).
- spark.kryo.registrator
+ spark.extraListeners
(none)
- If you use Kryo serialization, set this class to register your custom classes with Kryo. This
- property is useful if you need to register your classes in a custom way, e.g. to specify a custom
- field serializer. Otherwise spark.kryo.classesToRegister
is simpler. It should be
- set to a class that extends
-
- KryoRegistrator
.
- See the tuning guide for more details.
+ A comma-separated list of classes that implement SparkListener
; when initializing
+ SparkContext, instances of these classes will be created and registered with Spark's listener
+ bus. If a class has a single-argument constructor that accepts a SparkConf, that constructor
+ will be called; otherwise, a zero-argument constructor will be called. If no valid constructor
+ can be found, the SparkContext creation will fail with an exception.
@@ -190,6 +165,14 @@ of the most common options to set are:
Logs the effective SparkConf as INFO when a SparkContext is started.
+
+ spark.master
+ (none)
+
+ The cluster manager to connect to. See the list of
+ allowed master URL's.
+
+
Apart from these, the following properties are also available, and may be useful in some situations:
@@ -198,17 +181,27 @@ Apart from these, the following properties are also available, and may be useful
Property Name Default Meaning
- spark.driver.extraJavaOptions
+ spark.driver.extraClassPath
(none)
- A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.
+ Extra classpath entries to append to the classpath of the driver.
+
+
Note: In client mode, this config must not be set through the SparkConf
+ directly in your application, because the driver JVM has already started at that point.
+ Instead, please set this through the --driver-class-path
command line option or in
+ your default properties file.
- spark.driver.extraClassPath
+ spark.driver.extraJavaOptions
(none)
- Extra classpath entries to append to the classpath of the driver.
+ A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.
+
+
Note: In client mode, this config must not be set through the SparkConf
+ directly in your application, because the driver JVM has already started at that point.
+ Instead, please set this through the --driver-java-options
command line option or in
+ your default properties file.
@@ -216,54 +209,56 @@ Apart from these, the following properties are also available, and may be useful
(none)
Set a special library path to use when launching the driver JVM.
+
+
Note: In client mode, this config must not be set through the SparkConf
+ directly in your application, because the driver JVM has already started at that point.
+ Instead, please set this through the --driver-library-path
command line option or in
+ your default properties file.
- spark.executor.extraJavaOptions
- (none)
+ spark.driver.userClassPathFirst
+ false
- A string of extra JVM options to pass to executors. For instance, GC settings or other
- logging. Note that it is illegal to set Spark properties or heap size settings with this
- option. Spark properties should be set using a SparkConf object or the
- spark-defaults.conf file used with the spark-submit script. Heap size settings can be set
- with spark.executor.memory.
+ (Experimental) Whether to give user-added jars precedence over Spark's own jars when loading
+ classes in the the driver. This feature can be used to mitigate conflicts between Spark's
+ dependencies and user dependencies. It is currently an experimental feature.
+
+ This is used in cluster mode only.
spark.executor.extraClassPath
(none)
- Extra classpath entries to append to the classpath of executors. This exists primarily
- for backwards-compatibility with older versions of Spark. Users typically should not need
- to set this option.
+ Extra classpath entries to append to the classpath of executors. This exists primarily for
+ backwards-compatibility with older versions of Spark. Users typically should not need to set
+ this option.
- spark.executor.extraLibraryPath
+ spark.executor.extraJavaOptions
(none)
- Set a special library path to use when launching executor JVM's.
+ A string of extra JVM options to pass to executors. For instance, GC settings or other logging.
+ Note that it is illegal to set Spark properties or heap size settings with this option. Spark
+ properties should be set using a SparkConf object or the spark-defaults.conf file used with the
+ spark-submit script. Heap size settings can be set with spark.executor.memory.
- spark.executor.logs.rolling.strategy
+ spark.executor.extraLibraryPath
(none)
- Set the strategy of rolling of executor logs. By default it is disabled. It can
- be set to "time" (time-based rolling) or "size" (size-based rolling). For "time",
- use spark.executor.logs.rolling.time.interval
to set the rolling interval.
- For "size", use spark.executor.logs.rolling.size.maxBytes
to set
- the maximum file size for rolling.
+ Set a special library path to use when launching executor JVM's.
- spark.executor.logs.rolling.time.interval
- daily
+ spark.executor.logs.rolling.maxRetainedFiles
+ (none)
- Set the time interval by which the executor logs will be rolled over.
- Rolling is disabled by default. Valid values are `daily`, `hourly`, `minutely` or
- any interval in seconds. See spark.executor.logs.rolling.maxRetainedFiles
- for automatic cleaning of old logs.
+ Sets the number of latest rolling log files that are going to be retained by the system.
+ Older log files will be deleted. Disabled by default.
@@ -277,30 +272,40 @@ Apart from these, the following properties are also available, and may be useful
- spark.executor.logs.rolling.maxRetainedFiles
+ spark.executor.logs.rolling.strategy
(none)
- Sets the number of latest rolling log files that are going to be retained by the system.
- Older log files will be deleted. Disabled by default.
+ Set the strategy of rolling of executor logs. By default it is disabled. It can
+ be set to "time" (time-based rolling) or "size" (size-based rolling). For "time",
+ use spark.executor.logs.rolling.time.interval
to set the rolling interval.
+ For "size", use spark.executor.logs.rolling.size.maxBytes
to set
+ the maximum file size for rolling.
+
+
+
+ spark.executor.logs.rolling.time.interval
+ daily
+
+ Set the time interval by which the executor logs will be rolled over.
+ Rolling is disabled by default. Valid values are `daily`, `hourly`, `minutely` or
+ any interval in seconds. See spark.executor.logs.rolling.maxRetainedFiles
+ for automatic cleaning of old logs.
- spark.files.userClassPathFirst
+ spark.executor.userClassPathFirst
false
- (Experimental) Whether to give user-added jars precedence over Spark's own jars when
- loading classes in Executors. This feature can be used to mitigate conflicts between
- Spark's dependencies and user dependencies. It is currently an experimental feature.
- (Currently, this setting does not work for YARN, see SPARK-2996 for more details).
+ (Experimental) Same functionality as spark.driver.userClassPathFirst
, but
+ applied to executor instances.
- spark.python.worker.memory
- 512m
+ spark.executorEnv.[EnvironmentVariableName]
+ (none)
- Amount of memory to use per python worker process during aggregation, in the same
- format as JVM memory strings (e.g. 512m
, 2g
). If the memory
- used during aggregation goes above this amount, it will spill the data into disks.
+ Add the environment variable specified by EnvironmentVariableName
to the Executor
+ process. The user can specify multiple of these to set multiple environment variables.
@@ -326,6 +331,15 @@ Apart from these, the following properties are also available, and may be useful
automatically.
+
+ spark.python.worker.memory
+ 512m
+
+ Amount of memory to use per python worker process during aggregation, in the same
+ format as JVM memory strings (e.g. 512m
, 2g
). If the memory
+ used during aggregation goes above this amount, it will spill the data into disks.
+
+
spark.python.worker.reuse
true
@@ -336,40 +350,38 @@ Apart from these, the following properties are also available, and may be useful
from JVM to Python worker for every task.
+
+
+#### Shuffle Behavior
+
+Property Name Default Meaning
- spark.executorEnv.[EnvironmentVariableName]
- (none)
+ spark.reducer.maxMbInFlight
+ 48
- Add the environment variable specified by EnvironmentVariableName
to the Executor
- process. The user can specify multiple of these to set multiple environment variables.
+ Maximum size (in megabytes) of map outputs to fetch simultaneously from each reduce task. Since
+ each output requires us to create a buffer to receive it, this represents a fixed memory
+ overhead per reduce task, so keep it small unless you have a large amount of memory.
- spark.mesos.executor.home
- driver side SPARK_HOME
+ spark.shuffle.blockTransferService
+ netty
- Set the directory in which Spark is installed on the executors in Mesos. By default, the
- executors will simply use the driver's Spark home directory, which may not be visible to
- them. Note that this is only relevant if a Spark binary package is not specified through
- spark.executor.uri
.
+ Implementation to use for transferring shuffle and cached blocks between executors. There
+ are two implementations available: netty
and nio
. Netty-based
+ block transfer is intended to be simpler but equally efficient and is the default option
+ starting in 1.2.
- spark.mesos.executor.memoryOverhead
- executor memory * 0.07, with minimum of 384
+ spark.shuffle.compress
+ true
- This value is an additive for spark.executor.memory
, specified in MiB,
- which is used to calculate the total Mesos task memory. A value of 384
- implies a 384MiB overhead. Additionally, there is a hard-coded 7% minimum
- overhead. The final overhead will be the larger of either
- `spark.mesos.executor.memoryOverhead` or 7% of `spark.executor.memory`.
+ Whether to compress map output files. Generally a good idea. Compression will use
+ spark.io.compression.codec
.
-
-
-#### Shuffle Behavior
-
-Property Name Default Meaning
spark.shuffle.consolidateFiles
false
@@ -381,55 +393,46 @@ Apart from these, the following properties are also available, and may be useful
- spark.shuffle.spill
- true
+ spark.shuffle.file.buffer.kb
+ 32
- If set to "true", limits the amount of memory used during reduces by spilling data out to disk.
- This spilling threshold is specified by spark.shuffle.memoryFraction
.
+ Size of the in-memory buffer for each shuffle file output stream, in kilobytes. These buffers
+ reduce the number of disk seeks and system calls made in creating intermediate shuffle files.
- spark.shuffle.spill.compress
- true
+ spark.shuffle.io.maxRetries
+ 3
- Whether to compress data spilled during shuffles. Compression will use
- spark.io.compression.codec
.
+ (Netty only) Fetches that fail due to IO-related exceptions are automatically retried if this is
+ set to a non-zero value. This retry logic helps stabilize large shuffles in the face of long GC
+ pauses or transient network connectivity issues.
- spark.shuffle.memoryFraction
- 0.2
+ spark.shuffle.io.numConnectionsPerPeer
+ 1
- Fraction of Java heap to use for aggregation and cogroups during shuffles, if
- spark.shuffle.spill
is true. At any given time, the collective size of
- all in-memory maps used for shuffles is bounded by this limit, beyond which the contents will
- begin to spill to disk. If spills are often, consider increasing this value at the expense of
- spark.storage.memoryFraction
.
+ (Netty only) Connections between hosts are reused in order to reduce connection buildup for
+ large clusters. For clusters with many hard disks and few hosts, this may result in insufficient
+ concurrency to saturate all disks, and so users may consider increasing this value.
- spark.shuffle.compress
+ spark.shuffle.io.preferDirectBufs
true
- Whether to compress map output files. Generally a good idea. Compression will use
- spark.io.compression.codec
.
-
-
-
- spark.shuffle.file.buffer.kb
- 32
-
- Size of the in-memory buffer for each shuffle file output stream, in kilobytes. These buffers
- reduce the number of disk seeks and system calls made in creating intermediate shuffle files.
+ (Netty only) Off-heap buffers are used to reduce garbage collection during shuffle and cache
+ block transfer. For environments where off-heap memory is tightly limited, users may wish to
+ turn this off to force all allocations from Netty to be on-heap.
- spark.reducer.maxMbInFlight
- 48
+ spark.shuffle.io.retryWait
+ 5
- Maximum size (in megabytes) of map outputs to fetch simultaneously from each reduce task. Since
- each output requires us to create a buffer to receive it, this represents a fixed memory
- overhead per reduce task, so keep it small unless you have a large amount of memory.
+ (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying
+ is simply maxRetries * retryWait
, by default 15 seconds.
@@ -441,6 +444,17 @@ Apart from these, the following properties are also available, and may be useful
the default option starting in 1.2.
+
+ spark.shuffle.memoryFraction
+ 0.2
+
+ Fraction of Java heap to use for aggregation and cogroups during shuffles, if
+ spark.shuffle.spill
is true. At any given time, the collective size of
+ all in-memory maps used for shuffles is bounded by this limit, beyond which the contents will
+ begin to spill to disk. If spills are often, consider increasing this value at the expense of
+ spark.storage.memoryFraction
.
+
+
spark.shuffle.sort.bypassMergeThreshold
200
@@ -450,13 +464,19 @@ Apart from these, the following properties are also available, and may be useful
- spark.shuffle.blockTransferService
- netty
+ spark.shuffle.spill
+ true
- Implementation to use for transferring shuffle and cached blocks between executors. There
- are two implementations available: netty
and nio
. Netty-based
- block transfer is intended to be simpler but equally efficient and is the default option
- starting in 1.2.
+ If set to "true", limits the amount of memory used during reduces by spilling data out to disk.
+ This spilling threshold is specified by spark.shuffle.memoryFraction
.
+
+
+
+ spark.shuffle.spill.compress
+ true
+
+ Whether to compress data spilled during shuffles. Compression will use
+ spark.io.compression.codec
.
@@ -465,26 +485,28 @@ Apart from these, the following properties are also available, and may be useful
Property Name Default Meaning
- spark.ui.port
- 4040
+ spark.eventLog.compress
+ false
- Port for your application's dashboard, which shows memory and workload data.
+ Whether to compress logged events, if spark.eventLog.enabled
is true.
- spark.ui.retainedStages
- 1000
+ spark.eventLog.dir
+ file:///tmp/spark-events
- How many stages the Spark UI and status APIs remember before garbage
- collecting.
+ Base directory in which Spark events are logged, if spark.eventLog.enabled
is true.
+ Within this base directory, Spark creates a sub-directory for each application, and logs the
+ events specific to the application in this directory. Users may want to set this to
+ a unified location like an HDFS directory so history files can be read by the history server.
- spark.ui.retainedJobs
- 1000
+ spark.eventLog.enabled
+ false
- How many jobs the Spark UI and status APIs remember before garbage
- collecting.
+ Whether to log Spark events, useful for reconstructing the Web UI after the application has
+ finished.
@@ -495,28 +517,26 @@ Apart from these, the following properties are also available, and may be useful
- spark.eventLog.enabled
- false
+ spark.ui.port
+ 4040
- Whether to log Spark events, useful for reconstructing the Web UI after the application has
- finished.
+ Port for your application's dashboard, which shows memory and workload data.
- spark.eventLog.compress
- false
+ spark.ui.retainedJobs
+ 1000
- Whether to compress logged events, if spark.eventLog.enabled
is true.
+ How many jobs the Spark UI and status APIs remember before garbage
+ collecting.
- spark.eventLog.dir
- file:///tmp/spark-events
+ spark.ui.retainedStages
+ 1000
- Base directory in which Spark events are logged, if spark.eventLog.enabled
is true.
- Within this base directory, Spark creates a sub-directory for each application, and logs the
- events specific to the application in this directory. Users may want to set this to
- a unified location like an HDFS directory so history files can be read by the history server.
+ How many stages the Spark UI and status APIs remember before garbage
+ collecting.
@@ -532,12 +552,10 @@ Apart from these, the following properties are also available, and may be useful
- spark.rdd.compress
- false
+ spark.closure.serializer
+ org.apache.spark.serializer.
JavaSerializer
- Whether to compress serialized RDD partitions (e.g. for
- StorageLevel.MEMORY_ONLY_SER
). Can save substantial space at the cost of some
- extra CPU time.
+ Serializer class to use for closures. Currently only the Java serializer is supported.
@@ -553,14 +571,6 @@ Apart from these, the following properties are also available, and may be useful
and org.apache.spark.io.SnappyCompressionCodec
.
-
- spark.io.compression.snappy.block.size
- 32768
-
- Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec
- is used. Lowering this block size will also lower shuffle memory usage when Snappy is used.
-
-
spark.io.compression.lz4.block.size
32768
@@ -570,21 +580,20 @@ Apart from these, the following properties are also available, and may be useful
- spark.closure.serializer
- org.apache.spark.serializer.
JavaSerializer
+ spark.io.compression.snappy.block.size
+ 32768
- Serializer class to use for closures. Currently only the Java serializer is supported.
+ Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec
+ is used. Lowering this block size will also lower shuffle memory usage when Snappy is used.
- spark.serializer.objectStreamReset
- 100
+ spark.kryo.classesToRegister
+ (none)
- When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches
- objects to prevent writing redundant data, however that stops garbage collection of those
- objects. By calling 'reset' you flush that info from the serializer, and allow old
- objects to be collected. To turn off this periodic reset set it to -1.
- By default it will reset the serializer every 100 objects.
+ If you use Kryo serialization, give a comma-separated list of custom class names to register
+ with Kryo.
+ See the tuning guide for more details.
@@ -609,12 +618,16 @@ Apart from these, the following properties are also available, and may be useful
- spark.kryoserializer.buffer.mb
- 0.064
+ spark.kryo.registrator
+ (none)
- Initial size of Kryo's serialization buffer, in megabytes. Note that there will be one buffer
- per core on each worker. This buffer will grow up to
- spark.kryoserializer.buffer.max.mb
if needed.
+ If you use Kryo serialization, set this class to register your custom classes with Kryo. This
+ property is useful if you need to register your classes in a custom way, e.g. to specify a custom
+ field serializer. Otherwise spark.kryo.classesToRegister
is simpler. It should be
+ set to a class that extends
+
+ KryoRegistrator
.
+ See the tuning guide for more details.
@@ -626,11 +639,80 @@ Apart from these, the following properties are also available, and may be useful
inside Kryo.
+
+ spark.kryoserializer.buffer.mb
+ 0.064
+
+ Initial size of Kryo's serialization buffer, in megabytes. Note that there will be one buffer
+ per core on each worker. This buffer will grow up to
+ spark.kryoserializer.buffer.max.mb
if needed.
+
+
+
+ spark.rdd.compress
+ false
+
+ Whether to compress serialized RDD partitions (e.g. for
+ StorageLevel.MEMORY_ONLY_SER
). Can save substantial space at the cost of some
+ extra CPU time.
+
+
+
+ spark.serializer
+ org.apache.spark.serializer.
JavaSerializer
+
+ Class to use for serializing objects that will be sent over the network or need to be cached
+ in serialized form. The default of Java serialization works with any Serializable Java object
+ but is quite slow, so we recommend using
+ org.apache.spark.serializer.KryoSerializer
and configuring Kryo serialization
+ when speed is necessary. Can be any subclass of
+
+ org.apache.spark.Serializer
.
+
+
+
+ spark.serializer.objectStreamReset
+ 100
+
+ When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches
+ objects to prevent writing redundant data, however that stops garbage collection of those
+ objects. By calling 'reset' you flush that info from the serializer, and allow old
+ objects to be collected. To turn off this periodic reset set it to -1.
+ By default it will reset the serializer every 100 objects.
+
+
#### Execution Behavior
Property Name Default Meaning
+
+ spark.broadcast.blockSize
+ 4096
+
+ Size of each piece of a block in kilobytes for TorrentBroadcastFactory
.
+ Too large a value decreases parallelism during broadcast (makes it slower); however, if it is
+ too small, BlockManager
might take a performance hit.
+
+
+
+ spark.broadcast.factory
+ org.apache.spark.broadcast.
TorrentBroadcastFactory
+
+ Which broadcast implementation to use.
+
+
+
+ spark.cleaner.ttl
+ (infinite)
+
+ Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks
+ generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be
+ forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in
+ case of Spark Streaming applications). Note that any RDD that persists in memory for more than
+ this duration will be cleared as well.
+
+
spark.default.parallelism
@@ -649,19 +731,18 @@ Apart from these, the following properties are also available, and may be useful
- spark.broadcast.factory
- org.apache.spark.broadcast.
TorrentBroadcastFactory
-
- Which broadcast implementation to use.
-
+ spark.executor.heartbeatInterval
+ 10000
+ Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let
+ the driver know that the executor is still alive and update it with metrics for in-progress
+ tasks.
- spark.broadcast.blockSize
- 4096
+ spark.files.fetchTimeout
+ 60
- Size of each piece of a block in kilobytes for TorrentBroadcastFactory
.
- Too large a value decreases parallelism during broadcast (makes it slower); however, if it is
- too small, BlockManager
might take a performance hit.
+ Communication timeout to use when fetching files added through SparkContext.addFile() from
+ the driver, in seconds.
@@ -673,12 +754,23 @@ Apart from these, the following properties are also available, and may be useful
- spark.files.fetchTimeout
- 60
-
- Communication timeout to use when fetching files added through SparkContext.addFile() from
- the driver, in seconds.
-
+ spark.hadoop.cloneConf
+ false
+ If set to true, clones a new Hadoop Configuration
object for each task. This
+ option should be enabled to work around Configuration
thread-safety issues (see
+ SPARK-2546 for more details).
+ This is disabled by default in order to avoid unexpected performance regressions for jobs that
+ are not affected by these issues.
+
+
+ spark.hadoop.validateOutputSpecs
+ true
+ If set to true, validates the output specification (e.g. checking if the output directory already exists)
+ used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing
+ output directories. We recommend that users do not disable this except if trying to achieve compatibility with
+ previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand.
+ This setting is ignored for jobs generated through Spark Streaming's StreamingContext, since
+ data may need to be rewritten to pre-existing output directories during checkpoint recovery.
spark.storage.memoryFraction
@@ -689,6 +781,15 @@ Apart from these, the following properties are also available, and may be useful
increase it if you configure your own old generation size.
+
+ spark.storage.memoryMapThreshold
+ 2097152
+
+ Size of a block, in bytes, above which Spark memory maps when reading a block from disk.
+ This prevents Spark from memory mapping very small blocks. In general, memory
+ mapping has high overhead for blocks close to or below the page size of the operating system.
+
+
spark.storage.unrollFraction
0.2
@@ -707,15 +808,6 @@ Apart from these, the following properties are also available, and may be useful
directories on Tachyon file system.
-
- spark.storage.memoryMapThreshold
- 2097152
-
- Size of a block, in bytes, above which Spark memory maps when reading a block from disk.
- This prevents Spark from memory mapping very small blocks. In general, memory
- mapping has high overhead for blocks close to or below the page size of the operating system.
-
-
spark.tachyonStore.url
tachyon://localhost:19998
@@ -723,115 +815,53 @@ Apart from these, the following properties are also available, and may be useful
The URL of the underlying Tachyon file system in the TachyonStore.
-
- spark.cleaner.ttl
- (infinite)
-
- Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks
- generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be
- forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in
- case of Spark Streaming applications). Note that any RDD that persists in memory for more than
- this duration will be cleared as well.
-
-
-
- spark.hadoop.validateOutputSpecs
- true
- If set to true, validates the output specification (e.g. checking if the output directory already exists)
- used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing
- output directories. We recommend that users do not disable this except if trying to achieve compatibility with
- previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand.
- This setting is ignored for jobs generated through Spark Streaming's StreamingContext, since
- data may need to be rewritten to pre-existing output directories during checkpoint recovery.
-
-
- spark.hadoop.cloneConf
- false
- If set to true, clones a new Hadoop Configuration
object for each task. This
- option should be enabled to work around Configuration
thread-safety issues (see
- SPARK-2546 for more details).
- This is disabled by default in order to avoid unexpected performance regressions for jobs that
- are not affected by these issues.
-
-
- spark.executor.heartbeatInterval
- 10000
- Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let
- the driver know that the executor is still alive and update it with metrics for in-progress
- tasks.
-
#### Networking
Property Name Default Meaning
- spark.driver.host
- (local hostname)
-
- Hostname or IP address for the driver to listen on.
- This is used for communicating with the executors and the standalone Master.
-
-
-
- spark.driver.port
- (random)
-
- Port for the driver to listen on.
- This is used for communicating with the executors and the standalone Master.
-
-
-
- spark.fileserver.port
- (random)
-
- Port for the driver's HTTP file server to listen on.
-
-
-
- spark.broadcast.port
- (random)
-
- Port for the driver's HTTP broadcast server to listen on.
- This is not relevant for torrent broadcast.
-
-
-
- spark.replClassServer.port
- (random)
-
- Port for the driver's HTTP class server to listen on.
- This is only relevant for the Spark shell.
-
-
-
- spark.blockManager.port
- (random)
+ spark.akka.failure-detector.threshold
+ 300.0
- Port for all block managers to listen on. These exist on both the driver and the executors.
+ This is set to a larger value to disable failure detector that comes inbuilt akka. It can be
+ enabled again, if you plan to use this feature (Not recommended). This maps to akka's
+ `akka.remote.transport-failure-detector.threshold`. Tune this in combination of
+ `spark.akka.heartbeat.pauses` and `spark.akka.heartbeat.interval` if you need to.
- spark.executor.port
- (random)
+ spark.akka.frameSize
+ 10
- Port for the executor to listen on. This is used for communicating with the driver.
+ Maximum message size to allow in "control plane" communication (for serialized tasks and task
+ results), in MB. Increase this if your tasks need to send back large results to the driver
+ (e.g. using collect()
on a large dataset).
- spark.port.maxRetries
- 16
+ spark.akka.heartbeat.interval
+ 1000
- Default maximum number of retries when binding to a port before giving up.
+ This is set to a larger value to disable the transport failure detector that comes built in to
+ Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger
+ interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more
+ informative for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses`
+ if you need to. A likely positive use case for using failure detector would be: a sensistive
+ failure detector can help evict rogue executors quickly. However this is usually not the case
+ as GC pauses and network lags are expected in a real Spark cluster. Apart from that enabling
+ this leads to a lot of exchanges of heart beats between nodes leading to flooding the network
+ with those.
- spark.akka.frameSize
- 10
+ spark.akka.heartbeat.pauses
+ 6000
- Maximum message size to allow in "control plane" communication (for serialized tasks and task
- results), in MB. Increase this if your tasks need to send back large results to the driver
- (e.g. using collect()
on a large dataset).
+ This is set to a larger value to disable the transport failure detector that comes built in to Akka.
+ It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart
+ beat pause in seconds for Akka. This can be used to control sensitivity to GC pauses. Tune
+ this along with `spark.akka.heartbeat.interval` if you need to.
@@ -850,116 +880,80 @@ Apart from these, the following properties are also available, and may be useful
- spark.network.timeout
- 120
-
- Default timeout for all network interactions, in seconds. This config will be used in
- place of spark.core.connection.ack.wait.timeout
, spark.akka.timeout
,
- spark.storage.blockManagerSlaveTimeoutMs
or
- spark.shuffle.io.connectionTimeout
, if they are not configured.
-
-
-
- spark.akka.heartbeat.pauses
- 6000
-
- This is set to a larger value to disable failure detector that comes inbuilt akka. It can be
- enabled again, if you plan to use this feature (Not recommended). Acceptable heart beat pause
- in seconds for akka. This can be used to control sensitivity to gc pauses. Tune this in
- combination of `spark.akka.heartbeat.interval` and `spark.akka.failure-detector.threshold`
- if you need to.
-
-
-
- spark.akka.failure-detector.threshold
- 300.0
+ spark.blockManager.port
+ (random)
- This is set to a larger value to disable failure detector that comes inbuilt akka. It can be
- enabled again, if you plan to use this feature (Not recommended). This maps to akka's
- `akka.remote.transport-failure-detector.threshold`. Tune this in combination of
- `spark.akka.heartbeat.pauses` and `spark.akka.heartbeat.interval` if you need to.
+ Port for all block managers to listen on. These exist on both the driver and the executors.
- spark.akka.heartbeat.interval
- 1000
+ spark.broadcast.port
+ (random)
- This is set to a larger value to disable failure detector that comes inbuilt akka. It can be
- enabled again, if you plan to use this feature (Not recommended). A larger interval value in
- seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for
- akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and
- `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using
- failure detector can be, a sensistive failure detector can help evict rogue executors really
- quick. However this is usually not the case as gc pauses and network lags are expected in a
- real Spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats
- between nodes leading to flooding the network with those.
+ Port for the driver's HTTP broadcast server to listen on.
+ This is not relevant for torrent broadcast.
- spark.shuffle.io.preferDirectBufs
- true
+ spark.driver.host
+ (local hostname)
- (Netty only) Off-heap buffers are used to reduce garbage collection during shuffle and cache
- block transfer. For environments where off-heap memory is tightly limited, users may wish to
- turn this off to force all allocations from Netty to be on-heap.
+ Hostname or IP address for the driver to listen on.
+ This is used for communicating with the executors and the standalone Master.
- spark.shuffle.io.numConnectionsPerPeer
- 1
+ spark.driver.port
+ (random)
- (Netty only) Connections between hosts are reused in order to reduce connection buildup for
- large clusters. For clusters with many hard disks and few hosts, this may result in insufficient
- concurrency to saturate all disks, and so users may consider increasing this value.
+ Port for the driver to listen on.
+ This is used for communicating with the executors and the standalone Master.
- spark.shuffle.io.maxRetries
- 3
+ spark.executor.port
+ (random)
- (Netty only) Fetches that fail due to IO-related exceptions are automatically retried if this is
- set to a non-zero value. This retry logic helps stabilize large shuffles in the face of long GC
- pauses or transient network connectivity issues.
+ Port for the executor to listen on. This is used for communicating with the driver.
- spark.shuffle.io.retryWait
- 5
+ spark.fileserver.port
+ (random)
- (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying
- is simply maxRetries * retryWait
, by default 15 seconds.
+ Port for the driver's HTTP file server to listen on.
-
-
-#### Scheduling
-
-Property Name Default Meaning
- spark.task.cpus
- 1
+ spark.network.timeout
+ 120
- Number of cores to allocate for each task.
+ Default timeout for all network interactions, in seconds. This config will be used in
+ place of spark.core.connection.ack.wait.timeout
, spark.akka.timeout
,
+ spark.storage.blockManagerSlaveTimeoutMs
or
+ spark.shuffle.io.connectionTimeout
, if they are not configured.
- spark.task.maxFailures
- 4
+ spark.port.maxRetries
+ 16
- Number of individual task failures before giving up on the job.
- Should be greater than or equal to 1. Number of allowed retries = this value - 1.
+ Default maximum number of retries when binding to a port before giving up.
- spark.scheduler.mode
- FIFO
+ spark.replClassServer.port
+ (random)
- The scheduling mode between
- jobs submitted to the same SparkContext. Can be set to FAIR
- to use fair sharing instead of queueing jobs one after another. Useful for
- multi-user services.
+ Port for the driver's HTTP class server to listen on.
+ This is only relevant for the Spark shell.
+
+
+#### Scheduling
+
+Property Name Default Meaning
spark.cores.max
(not set)
@@ -973,43 +967,12 @@ Apart from these, the following properties are also available, and may be useful
- spark.mesos.coarse
- false
-
- If set to "true", runs over Mesos clusters in
- "coarse-grained" sharing mode,
- where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per
- Spark task. This gives lower-latency scheduling for short queries, but leaves resources in use
- for the whole duration of the Spark job.
-
-
-
- spark.speculation
+ spark.localExecution.enabled
false
- If set to "true", performs speculative execution of tasks. This means if one or more tasks are
- running slowly in a stage, they will be re-launched.
-
-
-
- spark.speculation.interval
- 100
-
- How often Spark will check for tasks to speculate, in milliseconds.
-
-
-
- spark.speculation.quantile
- 0.75
-
- Percentage of tasks which must be complete before speculation is enabled for a particular stage.
-
-
-
- spark.speculation.multiplier
- 1.5
-
- How many times slower a task is than the median to be considered for speculation.
+ Enables Spark to run certain jobs, such as first() or take() on the driver, without sending
+ tasks to the cluster. This can make certain jobs execute very quickly, but may require
+ shipping a whole partition of data to the driver.
@@ -1025,19 +988,19 @@ Apart from these, the following properties are also available, and may be useful
- spark.locality.wait.process
+ spark.locality.wait.node
spark.locality.wait
- Customize the locality wait for process locality. This affects tasks that attempt to access
- cached data in a particular executor process.
+ Customize the locality wait for node locality. For example, you can set this to 0 to skip
+ node locality and search immediately for rack locality (if your cluster has rack information).
- spark.locality.wait.node
+ spark.locality.wait.process
spark.locality.wait
- Customize the locality wait for node locality. For example, you can set this to 0 to skip
- node locality and search immediately for rack locality (if your cluster has rack information).
+ Customize the locality wait for process locality. This affects tasks that attempt to access
+ cached data in a particular executor process.
@@ -1048,16 +1011,16 @@ Apart from these, the following properties are also available, and may be useful
- spark.scheduler.revive.interval
- 1000
+ spark.scheduler.maxRegisteredResourcesWaitingTime
+ 30000
- The interval length for the scheduler to revive the worker resource offers to run tasks
+ Maximum amount of time to wait for resources to register before scheduling begins
(in milliseconds).
-
+
spark.scheduler.minRegisteredResourcesRatio
- 0.0 for Mesos and Standalone mode, 0.8 for YARN
+ 0.8 for YARN mode; 0.0 otherwise
The minimum ratio of registered resources (registered resources / total expected resources)
(resources are executors in yarn mode, CPU cores in standalone mode)
@@ -1068,25 +1031,70 @@ Apart from these, the following properties are also available, and may be useful
- spark.scheduler.maxRegisteredResourcesWaitingTime
- 30000
+ spark.scheduler.mode
+ FIFO
- Maximum amount of time to wait for resources to register before scheduling begins
+ The scheduling mode between
+ jobs submitted to the same SparkContext. Can be set to FAIR
+ to use fair sharing instead of queueing jobs one after another. Useful for
+ multi-user services.
+
+
+
+ spark.scheduler.revive.interval
+ 1000
+
+ The interval length for the scheduler to revive the worker resource offers to run tasks
(in milliseconds).
- spark.localExecution.enabled
+ spark.speculation
false
- Enables Spark to run certain jobs, such as first() or take() on the driver, without sending
- tasks to the cluster. This can make certain jobs execute very quickly, but may require
- shipping a whole partition of data to the driver.
+ If set to "true", performs speculative execution of tasks. This means if one or more tasks are
+ running slowly in a stage, they will be re-launched.
+
+
+
+ spark.speculation.interval
+ 100
+
+ How often Spark will check for tasks to speculate, in milliseconds.
+
+
+
+ spark.speculation.multiplier
+ 1.5
+
+ How many times slower a task is than the median to be considered for speculation.
+
+
+
+ spark.speculation.quantile
+ 0.75
+
+ Percentage of tasks which must be complete before speculation is enabled for a particular stage.
+
+
+
+ spark.task.cpus
+ 1
+
+ Number of cores to allocate for each task.
+
+
+
+ spark.task.maxFailures
+ 4
+
+ Number of individual task failures before giving up on the job.
+ Should be greater than or equal to 1. Number of allowed retries = this value - 1.
-#### Dynamic allocation
+#### Dynamic Allocation
Property Name Default Meaning
@@ -1098,29 +1106,46 @@ Apart from these, the following properties are also available, and may be useful
available on YARN mode. For more detail, see the description
here.
- This requires the following configurations to be set:
+ This requires spark.shuffle.service.enabled
to be set.
+ The following configurations are also relevant:
spark.dynamicAllocation.minExecutors
,
spark.dynamicAllocation.maxExecutors
, and
- spark.shuffle.service.enabled
+ spark.dynamicAllocation.initialExecutors
+
+
+
+ spark.dynamicAllocation.executorIdleTimeout
+ 600
+
+ If dynamic allocation is enabled and an executor has been idle for more than this duration
+ (in seconds), the executor will be removed. For more detail, see this
+ description.
+ spark.dynamicAllocation.initialExecutors
spark.dynamicAllocation.minExecutors
- (none)
- Lower bound for the number of executors if dynamic allocation is enabled (required).
+ Initial number of executors to run if dynamic allocation is enabled.
spark.dynamicAllocation.maxExecutors
- (none)
+ Integer.MAX_VALUE
+
+ Upper bound for the number of executors if dynamic allocation is enabled.
+
+
+
+ spark.dynamicAllocation.minExecutors
+ 0
- Upper bound for the number of executors if dynamic allocation is enabled (required).
+ Lower bound for the number of executors if dynamic allocation is enabled.
spark.dynamicAllocation.schedulerBacklogTimeout
- 60
+ 5
If dynamic allocation is enabled and there have been pending tasks backlogged for more than
this duration (in seconds), new executors will be requested. For more detail, see this
@@ -1136,20 +1161,30 @@ Apart from these, the following properties are also available, and may be useful
description.
-
- spark.dynamicAllocation.executorIdleTimeout
- 600
-
- If dynamic allocation is enabled and an executor has been idle for more than this duration
- (in seconds), the executor will be removed. For more detail, see this
- description.
-
-
#### Security
Property Name Default Meaning
+
+ spark.acls.enable
+ false
+
+ Whether Spark acls should are enabled. If enabled, this checks to see if the user has
+ access permissions to view or modify the job. Note this requires the user to be known,
+ so if the user comes across as null no checks are done. Filters can be used with the UI
+ to authenticate and set the user.
+
+
+
+ spark.admin.acls
+ Empty
+
+ Comma separated list of users/administrators that have view and modify access to all Spark jobs.
+ This can be used if you run on a shared cluster and have a set of administrators or devs who
+ help debug when things work.
+
+
spark.authenticate
false
@@ -1166,6 +1201,15 @@ Apart from these, the following properties are also available, and may be useful
not running on YARN and authentication is enabled.
+
+ spark.core.connection.ack.wait.timeout
+ 60
+
+ Number of seconds for the connection to wait for ack to occur before timing
+ out and giving up. To avoid unwilling timeout caused by long pause like GC,
+ you can set larger value.
+
+
spark.core.connection.auth.wait.timeout
30
@@ -1175,12 +1219,11 @@ Apart from these, the following properties are also available, and may be useful
- spark.core.connection.ack.wait.timeout
- 60
+ spark.modify.acls
+ Empty
- Number of seconds for the connection to wait for ack to occur before timing
- out and giving up. To avoid unwilling timeout caused by long pause like GC,
- you can set larger value.
+ Comma separated list of users that have modify access to the Spark job. By default only the
+ user that started the Spark job has access to modify it (kill it for example).
@@ -1197,16 +1240,6 @@ Apart from these, the following properties are also available, and may be useful
-Dspark.com.test.filter1.params='param1=foo,param2=testing'
-
- spark.acls.enable
- false
-
- Whether Spark acls should are enabled. If enabled, this checks to see if the user has
- access permissions to view or modify the job. Note this requires the user to be known,
- so if the user comes across as null no checks are done. Filters can be used with the UI
- to authenticate and set the user.
-
-
spark.ui.view.acls
Empty
@@ -1215,25 +1248,88 @@ Apart from these, the following properties are also available, and may be useful
user that started the Spark job has view access.
-
- spark.modify.acls
- Empty
-
- Comma separated list of users that have modify access to the Spark job. By default only the
- user that started the Spark job has access to modify it (kill it for example).
-
-
-
- spark.admin.acls
- Empty
-
- Comma separated list of users/administrators that have view and modify access to all Spark jobs.
- This can be used if you run on a shared cluster and have a set of administrators or devs who
- help debug when things work.
-
-
+#### Encryption
+
+
+ Property Name Default Meaning
+
+ spark.ssl.enabled
+ false
+
+ Whether to enable SSL connections on all supported protocols.
+
+ All the SSL settings like spark.ssl.xxx
where xxx
is a
+ particular configuration property, denote the global configuration for all the supported
+ protocols. In order to override the global configuration for the particular protocol,
+ the properties must be overwritten in the protocol-specific namespace.
+
+ Use spark.ssl.YYY.XXX
settings to overwrite the global configuration for
+ particular protocol denoted by YYY
. Currently YYY
can be
+ either akka
for Akka based connections or fs
for broadcast and
+ file server.
+
+
+
+ spark.ssl.enabledAlgorithms
+ Empty
+
+ A comma separated list of ciphers. The specified ciphers must be supported by JVM.
+ The reference list of protocols one can find on
+ this
+ page.
+
+
+
+ spark.ssl.keyPassword
+ None
+
+ A password to the private key in key-store.
+
+
+
+ spark.ssl.keyStore
+ None
+
+ A path to a key-store file. The path can be absolute or relative to the directory where
+ the component is started in.
+
+
+
+ spark.ssl.keyStorePassword
+ None
+
+ A password to the key-store.
+
+
+
+ spark.ssl.protocol
+ None
+
+ A protocol name. The protocol must be supported by JVM. The reference list of protocols
+ one can find on this
+ page.
+
+
+
+ spark.ssl.trustStore
+ None
+
+ A path to a trust-store file. The path can be absolute or relative to the directory
+ where the component is started in.
+
+
+
+ spark.ssl.trustStorePassword
+ None
+
+ A password to the trust-store.
+
+
+
+
+
#### Spark Streaming
Property Name Default Meaning
@@ -1249,9 +1345,9 @@ Apart from these, the following properties are also available, and may be useful
spark.streaming.receiver.maxRate
- infinite
+ not set
- Maximum number records per second at which each receiver will receive data.
+ Maximum rate (number of records per second) at which each receiver will receive data.
Effectively, each stream will consume at most this number of records per second.
Setting this configuration to 0 or a negative number will put no limit on the rate.
See the deployment guide
@@ -1279,15 +1375,27 @@ Apart from these, the following properties are also available, and may be useful
higher memory usage in Spark.
+
+ spark.streaming.kafka.maxRatePerPartition
+ not set
+
+ Maximum rate (number of records per second) at which data will be read from each Kafka
+ partition when using the new Kafka direct stream API. See the
+ Kafka Integration guide
+ for more details.
+
+
#### Cluster Managers
Each cluster manager in Spark has additional configuration options. Configurations
can be found on the pages for each mode:
- * [YARN](running-on-yarn.html#configuration)
- * [Mesos](running-on-mesos.html)
- * [Standalone Mode](spark-standalone.html#cluster-launch-scripts)
+##### [YARN](running-on-yarn.html#configuration)
+
+##### [Mesos](running-on-mesos.html#configuration)
+
+##### [Standalone Mode](spark-standalone.html#cluster-launch-scripts)
# Environment Variables
diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md
index d50f445d7ecc7..8c9a1e1262d8f 100644
--- a/docs/ec2-scripts.md
+++ b/docs/ec2-scripts.md
@@ -52,7 +52,7 @@ identify machines belonging to each cluster in the Amazon EC2 Console.
```bash
export AWS_SECRET_ACCESS_KEY=AaBbCcDdEeFGgHhIiJjKkLlMmNnOoPpQqRrSsTtU
export AWS_ACCESS_KEY_ID=ABCDEFG1234567890123
-./spark-ec2 --key-pair=awskey --identity-file=awskey.pem --region=us-west-1 --zone=us-west-1a --spark-version=1.1.0 launch my-spark-cluster
+./spark-ec2 --key-pair=awskey --identity-file=awskey.pem --region=us-west-1 --zone=us-west-1a launch my-spark-cluster
```
- After everything launches, check that the cluster scheduler is up and sees
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index e298c51f8a5b7..c601d793a2e9a 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -1,6 +1,8 @@
---
layout: global
-title: GraphX Programming Guide
+displayTitle: GraphX Programming Guide
+title: GraphX
+description: GraphX graph processing library guide for Spark SPARK_VERSION_SHORT
---
* This will become a table of contents (this text will be scraped).
@@ -536,7 +538,7 @@ val joinedGraph = graph.joinVertices(uniqueCosts,
## Neighborhood Aggregation
-A key step in may graph analytics tasks is aggregating information about the neighborhood of each
+A key step in many graph analytics tasks is aggregating information about the neighborhood of each
vertex.
For example, we might want to know the number of followers each user has or the average age of the
the followers of each user. Many iterative graph algorithms (e.g., PageRank, Shortest Path, and
@@ -632,7 +634,7 @@ avgAgeOfOlderFollowers.collect.foreach(println(_))
### Map Reduce Triplets Transition Guide (Legacy)
-In earlier versions of GraphX we neighborhood aggregation was accomplished using the
+In earlier versions of GraphX neighborhood aggregation was accomplished using the
[`mapReduceTriplets`][Graph.mapReduceTriplets] operator:
{% highlight scala %}
@@ -661,7 +663,7 @@ val graph: Graph[Int, Float] = ...
def msgFun(triplet: Triplet[Int, Float]): Iterator[(Int, String)] = {
Iterator((triplet.dstId, "Hi"))
}
-def reduceFun(a: Int, b: Int): Int = a + b
+def reduceFun(a: String, b: String): String = a + " " + b
val result = graph.mapReduceTriplets[String](msgFun, reduceFun)
{% endhighlight %}
@@ -672,7 +674,7 @@ val graph: Graph[Int, Float] = ...
def msgFun(triplet: EdgeContext[Int, Float, String]) {
triplet.sendToDst("Hi")
}
-def reduceFun(a: Int, b: Int): Int = a + b
+def reduceFun(a: String, b: String): String = a + " " + b
val result = graph.aggregateMessages[String](msgFun, reduceFun)
{% endhighlight %}
@@ -680,8 +682,8 @@ val result = graph.aggregateMessages[String](msgFun, reduceFun)
### Computing Degree Information
A common aggregation task is computing the degree of each vertex: the number of edges adjacent to
-each vertex. In the context of directed graphs it often necessary to know the in-degree, out-
-degree, and the total degree of each vertex. The [`GraphOps`][GraphOps] class contains a
+each vertex. In the context of directed graphs it is often necessary to know the in-degree,
+out-degree, and the total degree of each vertex. The [`GraphOps`][GraphOps] class contains a
collection of operators to compute the degrees of each vertex. For example in the following we
compute the max in, out, and total degrees:
diff --git a/docs/img/PIClusteringFiveCirclesInputsAndOutputs.png b/docs/img/PIClusteringFiveCirclesInputsAndOutputs.png
deleted file mode 100644
index ed9adad11d03a..0000000000000
Binary files a/docs/img/PIClusteringFiveCirclesInputsAndOutputs.png and /dev/null differ
diff --git a/docs/index.md b/docs/index.md
index 171d6ddad62f3..b5b016e34795e 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -1,6 +1,8 @@
---
layout: global
-title: Spark Overview
+displayTitle: Spark Overview
+title: Overview
+description: Apache Spark SPARK_VERSION_SHORT documentation homepage
---
Apache Spark is a fast and general-purpose cluster computing system.
@@ -72,7 +74,7 @@ options for deployment:
in all supported languages (Scala, Java, Python)
* Modules built on Spark:
* [Spark Streaming](streaming-programming-guide.html): processing real-time data streams
- * [Spark SQL](sql-programming-guide.html): support for structured data and relational queries
+ * [Spark SQL and DataFrames](sql-programming-guide.html): support for structured data and relational queries
* [MLlib](mllib-guide.html): built-in machine learning library
* [GraphX](graphx-programming-guide.html): Spark's new API for graph processing
* [Bagel (Pregel on Spark)](bagel-programming-guide.html): older, simple graph processing model
@@ -113,6 +115,8 @@ options for deployment:
* [Spark Homepage](http://spark.apache.org)
* [Spark Wiki](https://cwiki.apache.org/confluence/display/SPARK)
+* [Spark Community](http://spark.apache.org/community.html) resources, including local meetups
+* [StackOverflow tag `apache-spark`](http://stackoverflow.com/questions/tagged/apache-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, Spark Streaming, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/3/),
@@ -121,11 +125,3 @@ options for deployment:
* [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),
[Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples),
[Python]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python))
-
-# Community
-
-To get help using Spark or keep up with Spark development, sign up for the [user mailing list](http://spark.apache.org/mailing-lists.html).
-
-If you're in the San Francisco Bay Area, there's a regular [Spark meetup](http://www.meetup.com/spark-users/) every few weeks. Come by to meet the developers and other users.
-
-Finally, if you'd like to contribute code to Spark, read [how to contribute](contributing-to-spark.html).
diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md
index a5425eb3557b2..5295e351dd711 100644
--- a/docs/job-scheduling.md
+++ b/docs/job-scheduling.md
@@ -77,11 +77,10 @@ scheduling while sharing cluster resources efficiently.
### Configuration and Setup
All configurations used by this feature live under the `spark.dynamicAllocation.*` namespace.
-To enable this feature, your application must set `spark.dynamicAllocation.enabled` to `true` and
-provide lower and upper bounds for the number of executors through
-`spark.dynamicAllocation.minExecutors` and `spark.dynamicAllocation.maxExecutors`. Other relevant
-configurations are described on the [configurations page](configuration.html#dynamic-allocation)
-and in the subsequent sections in detail.
+To enable this feature, your application must set `spark.dynamicAllocation.enabled` to `true`.
+Other relevant configurations are described on the
+[configurations page](configuration.html#dynamic-allocation) and in the subsequent sections in
+detail.
Additionally, your application must use an external shuffle service. The purpose of the service is
to preserve the shuffle files written by executors so the executors can be safely removed (more
diff --git a/docs/ml-guide.md b/docs/ml-guide.md
index be178d7689fdd..da6aef7f14c4c 100644
--- a/docs/ml-guide.md
+++ b/docs/ml-guide.md
@@ -23,13 +23,13 @@ to `spark.ml`.
Spark ML standardizes APIs for machine learning algorithms to make it easier to combine multiple algorithms into a single pipeline, or workflow. This section covers the key concepts introduced by the Spark ML API.
-* **[ML Dataset](ml-guide.html#ml-dataset)**: Spark ML uses the [`SchemaRDD`](api/scala/index.html#org.apache.spark.sql.SchemaRDD) from Spark SQL as a dataset which can hold a variety of data types.
+* **[ML Dataset](ml-guide.html#ml-dataset)**: Spark ML uses the [`DataFrame`](api/scala/index.html#org.apache.spark.sql.DataFrame) from Spark SQL as a dataset which can hold a variety of data types.
E.g., a dataset could have different columns storing text, feature vectors, true labels, and predictions.
-* **[`Transformer`](ml-guide.html#transformers)**: A `Transformer` is an algorithm which can transform one `SchemaRDD` into another `SchemaRDD`.
+* **[`Transformer`](ml-guide.html#transformers)**: A `Transformer` is an algorithm which can transform one `DataFrame` into another `DataFrame`.
E.g., an ML model is a `Transformer` which transforms an RDD with features into an RDD with predictions.
-* **[`Estimator`](ml-guide.html#estimators)**: An `Estimator` is an algorithm which can be fit on a `SchemaRDD` to produce a `Transformer`.
+* **[`Estimator`](ml-guide.html#estimators)**: An `Estimator` is an algorithm which can be fit on a `DataFrame` to produce a `Transformer`.
E.g., a learning algorithm is an `Estimator` which trains on a dataset and produces a model.
* **[`Pipeline`](ml-guide.html#pipeline)**: A `Pipeline` chains multiple `Transformer`s and `Estimator`s together to specify an ML workflow.
@@ -39,20 +39,20 @@ E.g., a learning algorithm is an `Estimator` which trains on a dataset and produ
## ML Dataset
Machine learning can be applied to a wide variety of data types, such as vectors, text, images, and structured data.
-Spark ML adopts the [`SchemaRDD`](api/scala/index.html#org.apache.spark.sql.SchemaRDD) from Spark SQL in order to support a variety of data types under a unified Dataset concept.
+Spark ML adopts the [`DataFrame`](api/scala/index.html#org.apache.spark.sql.DataFrame) from Spark SQL in order to support a variety of data types under a unified Dataset concept.
-`SchemaRDD` supports many basic and structured types; see the [Spark SQL datatype reference](sql-programming-guide.html#spark-sql-datatype-reference) for a list of supported types.
-In addition to the types listed in the Spark SQL guide, `SchemaRDD` can use ML [`Vector`](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) types.
+`DataFrame` supports many basic and structured types; see the [Spark SQL datatype reference](sql-programming-guide.html#spark-sql-datatype-reference) for a list of supported types.
+In addition to the types listed in the Spark SQL guide, `DataFrame` can use ML [`Vector`](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) types.
-A `SchemaRDD` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples.
+A `DataFrame` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples.
-Columns in a `SchemaRDD` are named. The code examples below use names such as "text," "features," and "label."
+Columns in a `DataFrame` are named. The code examples below use names such as "text," "features," and "label."
## ML Algorithms
### Transformers
-A [`Transformer`](api/scala/index.html#org.apache.spark.ml.Transformer) is an abstraction which includes feature transformers and learned models. Technically, a `Transformer` implements a method `transform()` which converts one `SchemaRDD` into another, generally by appending one or more columns.
+A [`Transformer`](api/scala/index.html#org.apache.spark.ml.Transformer) is an abstraction which includes feature transformers and learned models. Technically, a `Transformer` implements a method `transform()` which converts one `DataFrame` into another, generally by appending one or more columns.
For example:
* A feature transformer might take a dataset, read a column (e.g., text), convert it into a new column (e.g., feature vectors), append the new column to the dataset, and output the updated dataset.
@@ -60,7 +60,7 @@ For example:
### Estimators
-An [`Estimator`](api/scala/index.html#org.apache.spark.ml.Estimator) abstracts the concept of a learning algorithm or any algorithm which fits or trains on data. Technically, an `Estimator` implements a method `fit()` which accepts a `SchemaRDD` and produces a `Transformer`.
+An [`Estimator`](api/scala/index.html#org.apache.spark.ml.Estimator) abstracts the concept of a learning algorithm or any algorithm which fits or trains on data. Technically, an `Estimator` implements a method `fit()` which accepts a `DataFrame` and produces a `Transformer`.
For example, a learning algorithm such as `LogisticRegression` is an `Estimator`, and calling `fit()` trains a `LogisticRegressionModel`, which is a `Transformer`.
### Properties of ML Algorithms
@@ -101,7 +101,7 @@ We illustrate this for the simple text document workflow. The figure below is f
Above, the top row represents a `Pipeline` with three stages.
The first two (`Tokenizer` and `HashingTF`) are `Transformer`s (blue), and the third (`LogisticRegression`) is an `Estimator` (red).
-The bottom row represents data flowing through the pipeline, where cylinders indicate `SchemaRDD`s.
+The bottom row represents data flowing through the pipeline, where cylinders indicate `DataFrame`s.
The `Pipeline.fit()` method is called on the original dataset which has raw text documents and labels.
The `Tokenizer.transform()` method splits the raw text documents into words, adding a new column with words into the dataset.
The `HashingTF.transform()` method converts the words column into feature vectors, adding a new column with those vectors to the dataset.
@@ -130,7 +130,7 @@ Each stage's `transform()` method updates the dataset and passes it to the next
*DAG `Pipeline`s*: A `Pipeline`'s stages are specified as an ordered array. The examples given here are all for linear `Pipeline`s, i.e., `Pipeline`s in which each stage uses data produced by the previous stage. It is possible to create non-linear `Pipeline`s as long as the data flow graph forms a Directed Acyclic Graph (DAG). This graph is currently specified implicitly based on the input and output column names of each stage (generally specified as parameters). If the `Pipeline` forms a DAG, then the stages must be specified in topological order.
-*Runtime checking*: Since `Pipeline`s can operate on datasets with varied types, they cannot use compile-time type checking. `Pipeline`s and `PipelineModel`s instead do runtime checking before actually running the `Pipeline`. This type checking is done using the dataset *schema*, a description of the data types of columns in the `SchemaRDD`.
+*Runtime checking*: Since `Pipeline`s can operate on datasets with varied types, they cannot use compile-time type checking. `Pipeline`s and `PipelineModel`s instead do runtime checking before actually running the `Pipeline`. This type checking is done using the dataset *schema*, a description of the data types of columns in the `DataFrame`.
## Parameters
@@ -171,12 +171,12 @@ import org.apache.spark.sql.{Row, SQLContext}
val conf = new SparkConf().setAppName("SimpleParamsExample")
val sc = new SparkContext(conf)
val sqlContext = new SQLContext(sc)
-import sqlContext._
+import sqlContext.implicits._
// Prepare training data.
// We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes
-// into SchemaRDDs, where it uses the case class metadata to infer the schema.
-val training = sparkContext.parallelize(Seq(
+// into DataFrames, where it uses the case class metadata to infer the schema.
+val training = sc.parallelize(Seq(
LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)),
LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)),
LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)),
@@ -192,7 +192,7 @@ lr.setMaxIter(10)
.setRegParam(0.01)
// Learn a LogisticRegression model. This uses the parameters stored in lr.
-val model1 = lr.fit(training)
+val model1 = lr.fit(training.toDF)
// Since model1 is a Model (i.e., a Transformer produced by an Estimator),
// we can view the parameters it used during fit().
// This prints the parameter (name: value) pairs, where names are unique IDs for this
@@ -203,33 +203,35 @@ println("Model 1 was fit using parameters: " + model1.fittingParamMap)
// which supports several methods for specifying parameters.
val paramMap = ParamMap(lr.maxIter -> 20)
paramMap.put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter.
-paramMap.put(lr.regParam -> 0.1, lr.threshold -> 0.5) // Specify multiple Params.
+paramMap.put(lr.regParam -> 0.1, lr.threshold -> 0.55) // Specify multiple Params.
// One can also combine ParamMaps.
-val paramMap2 = ParamMap(lr.scoreCol -> "probability") // Changes output column name.
+val paramMap2 = ParamMap(lr.probabilityCol -> "myProbability") // Change output column name
val paramMapCombined = paramMap ++ paramMap2
// Now learn a new model using the paramMapCombined parameters.
// paramMapCombined overrides all parameters set earlier via lr.set* methods.
-val model2 = lr.fit(training, paramMapCombined)
+val model2 = lr.fit(training.toDF, paramMapCombined)
println("Model 2 was fit using parameters: " + model2.fittingParamMap)
-// Prepare test documents.
-val test = sparkContext.parallelize(Seq(
+// Prepare test data.
+val test = sc.parallelize(Seq(
LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)),
LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)),
LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))))
-// Make predictions on test documents using the Transformer.transform() method.
+// Make predictions on test data using the Transformer.transform() method.
// LogisticRegression.transform will only use the 'features' column.
-// Note that model2.transform() outputs a 'probability' column instead of the usual 'score'
-// column since we renamed the lr.scoreCol parameter previously.
-model2.transform(test)
- .select('features, 'label, 'probability, 'prediction)
+// Note that model2.transform() outputs a 'myProbability' column instead of the usual
+// 'probability' column since we renamed the lr.probabilityCol parameter previously.
+model2.transform(test.toDF)
+ .select("features", "label", "myProbability", "prediction")
.collect()
- .foreach { case Row(features: Vector, label: Double, prob: Double, prediction: Double) =>
- println("(" + features + ", " + label + ") -> prob=" + prob + ", prediction=" + prediction)
+ .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) =>
+ println("($features, $label) -> prob=$prob, prediction=$prediction")
}
+
+sc.stop()
{% endhighlight %}
@@ -244,23 +246,23 @@ import org.apache.spark.ml.param.ParamMap;
import org.apache.spark.ml.classification.LogisticRegression;
import org.apache.spark.mllib.linalg.Vectors;
import org.apache.spark.mllib.regression.LabeledPoint;
-import org.apache.spark.sql.api.java.JavaSQLContext;
-import org.apache.spark.sql.api.java.JavaSchemaRDD;
-import org.apache.spark.sql.api.java.Row;
+import org.apache.spark.sql.DataFrame;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.Row;
SparkConf conf = new SparkConf().setAppName("JavaSimpleParamsExample");
JavaSparkContext jsc = new JavaSparkContext(conf);
-JavaSQLContext jsql = new JavaSQLContext(jsc);
+SQLContext jsql = new SQLContext(jsc);
// Prepare training data.
-// We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes
-// into SchemaRDDs, where it uses the case class metadata to infer the schema.
+// We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans
+// into DataFrames, where it uses the bean metadata to infer the schema.
List localTraining = Lists.newArrayList(
new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)),
new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)),
new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)),
new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)));
-JavaSchemaRDD training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class);
+DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledPoint.class);
// Create a LogisticRegression instance. This instance is an Estimator.
LogisticRegression lr = new LogisticRegression();
@@ -281,13 +283,13 @@ System.out.println("Model 1 was fit using parameters: " + model1.fittingParamMap
// We may alternatively specify parameters using a ParamMap.
ParamMap paramMap = new ParamMap();
-paramMap.put(lr.maxIter(), 20); // Specify 1 Param.
+paramMap.put(lr.maxIter().w(20)); // Specify 1 Param.
paramMap.put(lr.maxIter(), 30); // This overwrites the original maxIter.
-paramMap.put(lr.regParam(), 0.1);
+paramMap.put(lr.regParam().w(0.1), lr.threshold().w(0.55)); // Specify multiple Params.
// One can also combine ParamMaps.
ParamMap paramMap2 = new ParamMap();
-paramMap2.put(lr.scoreCol(), "probability"); // Changes output column name.
+paramMap2.put(lr.probabilityCol().w("myProbability")); // Change output column name
ParamMap paramMapCombined = paramMap.$plus$plus(paramMap2);
// Now learn a new model using the paramMapCombined parameters.
@@ -300,19 +302,19 @@ List localTest = Lists.newArrayList(
new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)),
new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)),
new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)));
-JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class);
+DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class);
// Make predictions on test documents using the Transformer.transform() method.
// LogisticRegression.transform will only use the 'features' column.
-// Note that model2.transform() outputs a 'probability' column instead of the usual 'score'
-// column since we renamed the lr.scoreCol parameter previously.
-model2.transform(test).registerAsTable("results");
-JavaSchemaRDD results =
- jsql.sql("SELECT features, label, probability, prediction FROM results");
-for (Row r: results.collect()) {
+// Note that model2.transform() outputs a 'myProbability' column instead of the usual
+// 'probability' column since we renamed the lr.probabilityCol parameter previously.
+DataFrame results = model2.transform(test);
+for (Row r: results.select("features", "label", "myProbability", "prediction").collect()) {
System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2)
+ ", prediction=" + r.get(3));
}
+
+jsc.stop();
{% endhighlight %}
@@ -330,6 +332,7 @@ import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.ml.Pipeline
import org.apache.spark.ml.classification.LogisticRegression
import org.apache.spark.ml.feature.{HashingTF, Tokenizer}
+import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.sql.{Row, SQLContext}
// Labeled and unlabeled instance types.
@@ -337,14 +340,14 @@ import org.apache.spark.sql.{Row, SQLContext}
case class LabeledDocument(id: Long, text: String, label: Double)
case class Document(id: Long, text: String)
-// Set up contexts. Import implicit conversions to SchemaRDD from sqlContext.
+// Set up contexts. Import implicit conversions to DataFrame from sqlContext.
val conf = new SparkConf().setAppName("SimpleTextClassificationPipeline")
val sc = new SparkContext(conf)
val sqlContext = new SQLContext(sc)
-import sqlContext._
+import sqlContext.implicits._
// Prepare training documents, which are labeled.
-val training = sparkContext.parallelize(Seq(
+val training = sc.parallelize(Seq(
LabeledDocument(0L, "a b c d e spark", 1.0),
LabeledDocument(1L, "b d", 0.0),
LabeledDocument(2L, "spark f g h", 1.0),
@@ -365,30 +368,32 @@ val pipeline = new Pipeline()
.setStages(Array(tokenizer, hashingTF, lr))
// Fit the pipeline to training documents.
-val model = pipeline.fit(training)
+val model = pipeline.fit(training.toDF)
// Prepare test documents, which are unlabeled.
-val test = sparkContext.parallelize(Seq(
+val test = sc.parallelize(Seq(
Document(4L, "spark i j k"),
Document(5L, "l m n"),
Document(6L, "mapreduce spark"),
Document(7L, "apache hadoop")))
// Make predictions on test documents.
-model.transform(test)
- .select('id, 'text, 'score, 'prediction)
+model.transform(test.toDF)
+ .select("id", "text", "probability", "prediction")
.collect()
- .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) =>
- println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction)
+ .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) =>
+ println("($id, $text) --> prob=$prob, prediction=$prediction")
}
+
+sc.stop()
{% endhighlight %}
{% highlight java %}
-import java.io.Serializable;
import java.util.List;
import com.google.common.collect.Lists;
+import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.ml.Pipeline;
import org.apache.spark.ml.PipelineModel;
@@ -396,10 +401,9 @@ import org.apache.spark.ml.PipelineStage;
import org.apache.spark.ml.classification.LogisticRegression;
import org.apache.spark.ml.feature.HashingTF;
import org.apache.spark.ml.feature.Tokenizer;
-import org.apache.spark.sql.api.java.JavaSQLContext;
-import org.apache.spark.sql.api.java.JavaSchemaRDD;
-import org.apache.spark.sql.api.java.Row;
-import org.apache.spark.SparkConf;
+import org.apache.spark.sql.DataFrame;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
// Labeled and unlabeled instance types.
// Spark SQL can infer schema from Java Beans.
@@ -434,7 +438,7 @@ public class LabeledDocument extends Document implements Serializable {
// Set up contexts.
SparkConf conf = new SparkConf().setAppName("JavaSimpleTextClassificationPipeline");
JavaSparkContext jsc = new JavaSparkContext(conf);
-JavaSQLContext jsql = new JavaSQLContext(jsc);
+SQLContext jsql = new SQLContext(jsc);
// Prepare training documents, which are labeled.
List localTraining = Lists.newArrayList(
@@ -442,8 +446,7 @@ List localTraining = Lists.newArrayList(
new LabeledDocument(1L, "b d", 0.0),
new LabeledDocument(2L, "spark f g h", 1.0),
new LabeledDocument(3L, "hadoop mapreduce", 0.0));
-JavaSchemaRDD training =
- jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class);
+DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class);
// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr.
Tokenizer tokenizer = new Tokenizer()
@@ -468,16 +471,62 @@ List localTest = Lists.newArrayList(
new Document(5L, "l m n"),
new Document(6L, "mapreduce spark"),
new Document(7L, "apache hadoop"));
-JavaSchemaRDD test =
- jsql.applySchema(jsc.parallelize(localTest), Document.class);
+DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class);
// Make predictions on test documents.
-model.transform(test).registerAsTable("prediction");
-JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction");
-for (Row r: predictions.collect()) {
- System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2)
+DataFrame predictions = model.transform(test);
+for (Row r: predictions.select("id", "text", "probability", "prediction").collect()) {
+ System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2)
+ ", prediction=" + r.get(3));
}
+
+jsc.stop();
+{% endhighlight %}
+
+
+
+{% highlight python %}
+from pyspark import SparkContext
+from pyspark.ml import Pipeline
+from pyspark.ml.classification import LogisticRegression
+from pyspark.ml.feature import HashingTF, Tokenizer
+from pyspark.sql import Row, SQLContext
+
+sc = SparkContext(appName="SimpleTextClassificationPipeline")
+sqlCtx = SQLContext(sc)
+
+# Prepare training documents, which are labeled.
+LabeledDocument = Row("id", "text", "label")
+training = sc.parallelize([(0L, "a b c d e spark", 1.0),
+ (1L, "b d", 0.0),
+ (2L, "spark f g h", 1.0),
+ (3L, "hadoop mapreduce", 0.0)]) \
+ .map(lambda x: LabeledDocument(*x)).toDF()
+
+# Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr.
+tokenizer = Tokenizer(inputCol="text", outputCol="words")
+hashingTF = HashingTF(inputCol=tokenizer.getOutputCol(), outputCol="features")
+lr = LogisticRegression(maxIter=10, regParam=0.01)
+pipeline = Pipeline(stages=[tokenizer, hashingTF, lr])
+
+# Fit the pipeline to training documents.
+model = pipeline.fit(training)
+
+# Prepare test documents, which are unlabeled.
+Document = Row("id", "text")
+test = sc.parallelize([(4L, "spark i j k"),
+ (5L, "l m n"),
+ (6L, "mapreduce spark"),
+ (7L, "apache hadoop")]) \
+ .map(lambda x: Document(*x)).toDF()
+
+# Make predictions on test documents and print columns of interest.
+prediction = model.transform(test)
+selected = prediction.select("id", "text", "prediction")
+for row in selected.collect():
+ print row
+
+sc.stop()
{% endhighlight %}
@@ -508,21 +557,21 @@ However, it is also a well-established method for choosing parameters which is m
{% highlight scala %}
import org.apache.spark.{SparkConf, SparkContext}
-import org.apache.spark.SparkContext._
import org.apache.spark.ml.Pipeline
import org.apache.spark.ml.classification.LogisticRegression
import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator
import org.apache.spark.ml.feature.{HashingTF, Tokenizer}
import org.apache.spark.ml.tuning.{ParamGridBuilder, CrossValidator}
+import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.sql.{Row, SQLContext}
val conf = new SparkConf().setAppName("CrossValidatorExample")
val sc = new SparkContext(conf)
val sqlContext = new SQLContext(sc)
-import sqlContext._
+import sqlContext.implicits._
// Prepare training documents, which are labeled.
-val training = sparkContext.parallelize(Seq(
+val training = sc.parallelize(Seq(
LabeledDocument(0L, "a b c d e spark", 1.0),
LabeledDocument(1L, "b d", 0.0),
LabeledDocument(2L, "spark f g h", 1.0),
@@ -565,24 +614,24 @@ crossval.setEstimatorParamMaps(paramGrid)
crossval.setNumFolds(2) // Use 3+ in practice
// Run cross-validation, and choose the best set of parameters.
-val cvModel = crossval.fit(training)
-// Get the best LogisticRegression model (with the best set of parameters from paramGrid).
-val lrModel = cvModel.bestModel
+val cvModel = crossval.fit(training.toDF)
// Prepare test documents, which are unlabeled.
-val test = sparkContext.parallelize(Seq(
+val test = sc.parallelize(Seq(
Document(4L, "spark i j k"),
Document(5L, "l m n"),
Document(6L, "mapreduce spark"),
Document(7L, "apache hadoop")))
// Make predictions on test documents. cvModel uses the best model found (lrModel).
-cvModel.transform(test)
- .select('id, 'text, 'score, 'prediction)
+cvModel.transform(test.toDF)
+ .select("id", "text", "probability", "prediction")
.collect()
- .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) =>
- println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction)
+ .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) =>
+ println(s"($id, $text) --> prob=$prob, prediction=$prediction")
}
+
+sc.stop()
{% endhighlight %}
@@ -592,7 +641,6 @@ import java.util.List;
import com.google.common.collect.Lists;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.ml.Model;
import org.apache.spark.ml.Pipeline;
import org.apache.spark.ml.PipelineStage;
import org.apache.spark.ml.classification.LogisticRegression;
@@ -603,13 +651,13 @@ import org.apache.spark.ml.param.ParamMap;
import org.apache.spark.ml.tuning.CrossValidator;
import org.apache.spark.ml.tuning.CrossValidatorModel;
import org.apache.spark.ml.tuning.ParamGridBuilder;
-import org.apache.spark.sql.api.java.JavaSQLContext;
-import org.apache.spark.sql.api.java.JavaSchemaRDD;
-import org.apache.spark.sql.api.java.Row;
+import org.apache.spark.sql.DataFrame;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
SparkConf conf = new SparkConf().setAppName("JavaCrossValidatorExample");
JavaSparkContext jsc = new JavaSparkContext(conf);
-JavaSQLContext jsql = new JavaSQLContext(jsc);
+SQLContext jsql = new SQLContext(jsc);
// Prepare training documents, which are labeled.
List localTraining = Lists.newArrayList(
@@ -625,8 +673,7 @@ List localTraining = Lists.newArrayList(
new LabeledDocument(9L, "a e c l", 0.0),
new LabeledDocument(10L, "spark compile", 1.0),
new LabeledDocument(11L, "hadoop software", 0.0));
-JavaSchemaRDD training =
- jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class);
+DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class);
// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr.
Tokenizer tokenizer = new Tokenizer()
@@ -660,8 +707,6 @@ crossval.setNumFolds(2); // Use 3+ in practice
// Run cross-validation, and choose the best set of parameters.
CrossValidatorModel cvModel = crossval.fit(training);
-// Get the best LogisticRegression model (with the best set of parameters from paramGrid).
-Model lrModel = cvModel.bestModel();
// Prepare test documents, which are unlabeled.
List localTest = Lists.newArrayList(
@@ -669,15 +714,16 @@ List localTest = Lists.newArrayList(
new Document(5L, "l m n"),
new Document(6L, "mapreduce spark"),
new Document(7L, "apache hadoop"));
-JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), Document.class);
+DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class);
// Make predictions on test documents. cvModel uses the best model found (lrModel).
-cvModel.transform(test).registerAsTable("prediction");
-JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction");
-for (Row r: predictions.collect()) {
- System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2)
+DataFrame predictions = cvModel.transform(test);
+for (Row r: predictions.select("id", "text", "probability", "prediction").collect()) {
+ System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2)
+ ", prediction=" + r.get(3));
}
+
+jsc.stop();
{% endhighlight %}
@@ -686,6 +732,21 @@ for (Row r: predictions.collect()) {
# Dependencies
Spark ML currently depends on MLlib and has the same dependencies.
-Please see the [MLlib Dependencies guide](mllib-guide.html#Dependencies) for more info.
+Please see the [MLlib Dependencies guide](mllib-guide.html#dependencies) for more info.
Spark ML also depends upon Spark SQL, but the relevant parts of Spark SQL do not bring additional dependencies.
+
+# Migration Guide
+
+## From 1.2 to 1.3
+
+The main API changes are from Spark SQL. We list the most important changes here:
+
+* The old [SchemaRDD](http://spark.apache.org/docs/1.2.1/api/scala/index.html#org.apache.spark.sql.SchemaRDD) has been replaced with [DataFrame](api/scala/index.html#org.apache.spark.sql.DataFrame) with a somewhat modified API. All algorithms in Spark ML which used to use SchemaRDD now use DataFrame.
+* In Spark 1.2, we used implicit conversions from `RDD`s of `LabeledPoint` into `SchemaRDD`s by calling `import sqlContext._` where `sqlContext` was an instance of `SQLContext`. These implicits have been moved, so we now call `import sqlContext.implicits._`.
+* Java APIs for SQL have also changed accordingly. Please see the examples above and the [Spark SQL Programming Guide](sql-programming-guide.html) for details.
+
+Other changes were in `LogisticRegression`:
+
+* The `scoreCol` output column (with default value "score") was renamed to be `probabilityCol` (with default value "probability"). The type was originally `Double` (for the probability of class 1.0), but it is now `Vector` (for the probability of each class, to support multiclass classification in the future).
+* In Spark 1.2, `LogisticRegressionModel` did not include an intercept. In Spark 1.3, it includes an intercept; however, it will always be 0.0 since it uses the default settings for [spark.mllib.LogisticRegressionWithLBFGS](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS). The option to use an intercept will be added in the future.
diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md
index 719cc95767b00..8e91d62f4a907 100644
--- a/docs/mllib-classification-regression.md
+++ b/docs/mllib-classification-regression.md
@@ -17,13 +17,13 @@ the supported algorithms for each type of problem.
- Binary Classification linear SVMs, logistic regression, decision trees, naive Bayes
+ Binary Classification linear SVMs, logistic regression, decision trees, random forests, gradient-boosted trees, naive Bayes
- Multiclass Classification decision trees, naive Bayes
+ Multiclass Classification decision trees, random forests, naive Bayes
- Regression linear least squares, Lasso, ridge regression, decision trees
+ Regression linear least squares, Lasso, ridge regression, decision trees, random forests, gradient-boosted trees, isotonic regression
@@ -34,4 +34,8 @@ More details for these methods can be found here:
* [binary classification (SVMs, logistic regression)](mllib-linear-methods.html#binary-classification)
* [linear regression (least squares, Lasso, ridge)](mllib-linear-methods.html#linear-least-squares-lasso-and-ridge-regression)
* [Decision trees](mllib-decision-tree.html)
+* [Ensembles of decision trees](mllib-ensembles.html)
+ * [random forests](mllib-ensembles.html#random-forests)
+ * [gradient-boosted trees](mllib-ensembles.html#gradient-boosted-trees-gbts)
* [Naive Bayes](mllib-naive-bayes.html)
+* [Isotonic regression](mllib-isotonic-regression.html)
diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md
index 413b824e369da..0b6db4fcb7b1f 100644
--- a/docs/mllib-clustering.md
+++ b/docs/mllib-clustering.md
@@ -4,25 +4,25 @@ title: Clustering - MLlib
displayTitle: MLlib - Clustering
---
-* Table of contents
-{:toc}
-
-
-## Clustering
-
Clustering is an unsupervised learning problem whereby we aim to group subsets
of entities with one another based on some notion of similarity. Clustering is
often used for exploratory analysis and/or as a component of a hierarchical
supervised learning pipeline (in which distinct classifiers or regression
-models are trained for each cluster).
+models are trained for each cluster).
+
+MLlib supports the following models:
-MLlib supports
-[k-means](http://en.wikipedia.org/wiki/K-means_clustering) clustering, one of
-the most commonly used clustering algorithms that clusters the data points into
+* Table of contents
+{:toc}
+
+## K-means
+
+[k-means](http://en.wikipedia.org/wiki/K-means_clustering) is one of the
+most commonly used clustering algorithms that clusters the data points into a
predefined number of clusters. The MLlib implementation includes a parallelized
variant of the [k-means++](http://en.wikipedia.org/wiki/K-means%2B%2B) method
called [kmeans||](http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf).
-The implementation in MLlib has the following parameters:
+The implementation in MLlib has the following parameters:
* *k* is the number of desired clusters.
* *maxIterations* is the maximum number of iterations to run.
@@ -32,29 +32,9 @@ initialization via k-means\|\|.
guaranteed to find a globally optimal solution, and when run multiple times on
a given dataset, the algorithm returns the best clustering result).
* *initializationSteps* determines the number of steps in the k-means\|\| algorithm.
-* *epsilon* determines the distance threshold within which we consider k-means to have converged.
-
-### Power Iteration Clustering
+* *epsilon* determines the distance threshold within which we consider k-means to have converged.
-Power iteration clustering is a scalable and efficient algorithm for clustering points given pointwise mutual affinity values. Internally the algorithm:
-
-* accepts a [Graph](https://spark.apache.org/docs/0.9.2/api/graphx/index.html#org.apache.spark.graphx.Graph) that represents a normalized pairwise affinity between all input points.
-* calculates the principal eigenvalue and eigenvector
-* Clusters each of the input points according to their principal eigenvector component value
-
-Details of this algorithm are found within [Power Iteration Clustering, Lin and Cohen]{www.icml2010.org/papers/387.pdf}
-
-Example outputs for a dataset inspired by the paper - but with five clusters instead of three- have he following output from our implementation:
-
-
-
-
-
-
-### Examples
+**Examples**
@@ -168,41 +148,370 @@ print("Within Set Sum of Squared Error = " + str(WSSSE))
-In order to run the above application, follow the instructions
-provided in the [Self-Contained Applications](quick-start.html#self-contained-applications)
-section of the Spark
-Quick Start guide. Be sure to also include *spark-mllib* to your build file as
-a dependency.
+## Gaussian mixture
+
+A [Gaussian Mixture Model](http://en.wikipedia.org/wiki/Mixture_model#Multivariate_Gaussian_mixture_model)
+represents a composite distribution whereby points are drawn from one of *k* Gaussian sub-distributions,
+each with its own probability. The MLlib implementation uses the
+[expectation-maximization](http://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm)
+ algorithm to induce the maximum-likelihood model given a set of samples. The implementation
+has the following parameters:
+
+* *k* is the number of desired clusters.
+* *convergenceTol* is the maximum change in log-likelihood at which we consider convergence achieved.
+* *maxIterations* is the maximum number of iterations to perform without reaching convergence.
+* *initialModel* is an optional starting point from which to start the EM algorithm. If this parameter is omitted, a random starting point will be constructed from the data.
+
+**Examples**
+
+
+
+In the following example after loading and parsing data, we use a
+[GaussianMixture](api/scala/index.html#org.apache.spark.mllib.clustering.GaussianMixture)
+object to cluster the data into two clusters. The number of desired clusters is passed
+to the algorithm. We then output the parameters of the mixture model.
+
+{% highlight scala %}
+import org.apache.spark.mllib.clustering.GaussianMixture
+import org.apache.spark.mllib.linalg.Vectors
+
+// Load and parse the data
+val data = sc.textFile("data/mllib/gmm_data.txt")
+val parsedData = data.map(s => Vectors.dense(s.trim.split(' ').map(_.toDouble))).cache()
+
+// Cluster the data into two classes using GaussianMixture
+val gmm = new GaussianMixture().setK(2).run(parsedData)
+
+// output parameters of max-likelihood model
+for (i <- 0 until gmm.k) {
+ println("weight=%f\nmu=%s\nsigma=\n%s\n" format
+ (gmm.weights(i), gmm.gaussians(i).mu, gmm.gaussians(i).sigma))
+}
+
+{% endhighlight %}
+
+
+
+All of MLlib's methods use Java-friendly types, so you can import and call them there the same
+way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the
+Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by
+calling `.rdd()` on your `JavaRDD` object. A self-contained application example
+that is equivalent to the provided example in Scala is given below:
+
+{% highlight java %}
+import org.apache.spark.api.java.*;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.mllib.clustering.GaussianMixture;
+import org.apache.spark.mllib.clustering.GaussianMixtureModel;
+import org.apache.spark.mllib.linalg.Vector;
+import org.apache.spark.mllib.linalg.Vectors;
+import org.apache.spark.SparkConf;
+
+public class GaussianMixtureExample {
+ public static void main(String[] args) {
+ SparkConf conf = new SparkConf().setAppName("GaussianMixture Example");
+ JavaSparkContext sc = new JavaSparkContext(conf);
+
+ // Load and parse data
+ String path = "data/mllib/gmm_data.txt";
+ JavaRDD data = sc.textFile(path);
+ JavaRDD parsedData = data.map(
+ new Function() {
+ public Vector call(String s) {
+ String[] sarray = s.trim().split(" ");
+ double[] values = new double[sarray.length];
+ for (int i = 0; i < sarray.length; i++)
+ values[i] = Double.parseDouble(sarray[i]);
+ return Vectors.dense(values);
+ }
+ }
+ );
+ parsedData.cache();
+
+ // Cluster the data into two classes using GaussianMixture
+ GaussianMixtureModel gmm = new GaussianMixture().setK(2).run(parsedData.rdd());
+
+ // Output the parameters of the mixture model
+ for(int j=0; j
+
+
+In the following example after loading and parsing data, we use a
+[GaussianMixture](api/python/pyspark.mllib.html#pyspark.mllib.clustering.GaussianMixture)
+object to cluster the data into two clusters. The number of desired clusters is passed
+to the algorithm. We then output the parameters of the mixture model.
-## Streaming clustering
+{% highlight python %}
+from pyspark.mllib.clustering import GaussianMixture
+from numpy import array
+
+# Load and parse the data
+data = sc.textFile("data/mllib/gmm_data.txt")
+parsedData = data.map(lambda line: array([float(x) for x in line.strip().split(' ')]))
+
+# Build the model (cluster the data)
+gmm = GaussianMixture.train(parsedData, 2)
-When data arrive in a stream, we may want to estimate clusters dynamically,
-updating them as new data arrive. MLlib provides support for streaming k-means clustering,
-with parameters to control the decay (or "forgetfulness") of the estimates. The algorithm
-uses a generalization of the mini-batch k-means update rule. For each batch of data, we assign
+# output parameters of model
+for i in range(2):
+ print ("weight = ", gmm.weights[i], "mu = ", gmm.gaussians[i].mu,
+ "sigma = ", gmm.gaussians[i].sigma.toArray())
+
+{% endhighlight %}
+
+
+
+
+## Power iteration clustering (PIC)
+
+Power iteration clustering (PIC) is a scalable and efficient algorithm for clustering vertices of a
+graph given pairwise similarties as edge properties,
+described in [Lin and Cohen, Power Iteration Clustering](http://www.icml2010.org/papers/387.pdf).
+It computes a pseudo-eigenvector of the normalized affinity matrix of the graph via
+[power iteration](http://en.wikipedia.org/wiki/Power_iteration) and uses it to cluster vertices.
+MLlib includes an implementation of PIC using GraphX as its backend.
+It takes an `RDD` of `(srcId, dstId, similarity)` tuples and outputs a model with the clustering assignments.
+The similarities must be nonnegative.
+PIC assumes that the similarity measure is symmetric.
+A pair `(srcId, dstId)` regardless of the ordering should appear at most once in the input data.
+If a pair is missing from input, their similarity is treated as zero.
+MLlib's PIC implementation takes the following (hyper-)parameters:
+
+* `k`: number of clusters
+* `maxIterations`: maximum number of power iterations
+* `initializationMode`: initialization model. This can be either "random", which is the default,
+ to use a random vector as vertex properties, or "degree" to use normalized sum similarities.
+
+**Examples**
+
+In the following, we show code snippets to demonstrate how to use PIC in MLlib.
+
+
+
+
+[`PowerIterationClustering`](api/scala/index.html#org.apache.spark.mllib.clustering.PowerIterationClustering)
+implements the PIC algorithm.
+It takes an `RDD` of `(srcId: Long, dstId: Long, similarity: Double)` tuples representing the
+affinity matrix.
+Calling `PowerIterationClustering.run` returns a
+[`PowerIterationClusteringModel`](api/scala/index.html#org.apache.spark.mllib.clustering.PowerIterationClusteringModel),
+which contains the computed clustering assignments.
+
+{% highlight scala %}
+import org.apache.spark.mllib.clustering.PowerIterationClustering
+import org.apache.spark.mllib.linalg.Vectors
+
+val similarities: RDD[(Long, Long, Double)] = ...
+
+val pic = new PowerIteartionClustering()
+ .setK(3)
+ .setMaxIterations(20)
+val model = pic.run(similarities)
+
+model.assignments.foreach { a =>
+ println(s"${a.id} -> ${a.cluster}")
+}
+{% endhighlight %}
+
+A full example that produces the experiment described in the PIC paper can be found under
+[`examples/`](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala).
+
+
+
+
+
+[`PowerIterationClustering`](api/java/org/apache/spark/mllib/clustering/PowerIterationClustering.html)
+implements the PIC algorithm.
+It takes an `JavaRDD` of `(srcId: Long, dstId: Long, similarity: Double)` tuples representing the
+affinity matrix.
+Calling `PowerIterationClustering.run` returns a
+[`PowerIterationClusteringModel`](api/java/org/apache/spark/mllib/clustering/PowerIterationClusteringModel.html)
+which contains the computed clustering assignments.
+
+{% highlight java %}
+import scala.Tuple2;
+import scala.Tuple3;
+
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.mllib.clustering.PowerIterationClustering;
+import org.apache.spark.mllib.clustering.PowerIterationClusteringModel;
+
+JavaRDD> similarities = ...
+
+PowerIterationClustering pic = new PowerIterationClustering()
+ .setK(2)
+ .setMaxIterations(10);
+PowerIterationClusteringModel model = pic.run(similarities);
+
+for (PowerIterationClustering.Assignment a: model.assignments().toJavaRDD().collect()) {
+ System.out.println(a.id() + " -> " + a.cluster());
+}
+{% endhighlight %}
+
+
+
+
+## Latent Dirichlet allocation (LDA)
+
+[Latent Dirichlet allocation (LDA)](http://en.wikipedia.org/wiki/Latent_Dirichlet_allocation)
+is a topic model which infers topics from a collection of text documents.
+LDA can be thought of as a clustering algorithm as follows:
+
+* Topics correspond to cluster centers, and documents correspond to examples (rows) in a dataset.
+* Topics and documents both exist in a feature space, where feature vectors are vectors of word counts.
+* Rather than estimating a clustering using a traditional distance, LDA uses a function based
+ on a statistical model of how text documents are generated.
+
+LDA takes in a collection of documents as vectors of word counts.
+It learns clustering using [expectation-maximization](http://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm)
+on the likelihood function. After fitting on the documents, LDA provides:
+
+* Topics: Inferred topics, each of which is a probability distribution over terms (words).
+* Topic distributions for documents: For each document in the training set, LDA gives a probability distribution over topics.
+
+LDA takes the following parameters:
+
+* `k`: Number of topics (i.e., cluster centers)
+* `maxIterations`: Limit on the number of iterations of EM used for learning
+* `docConcentration`: Hyperparameter for prior over documents' distributions over topics. Currently must be > 1, where larger values encourage smoother inferred distributions.
+* `topicConcentration`: Hyperparameter for prior over topics' distributions over terms (words). Currently must be > 1, where larger values encourage smoother inferred distributions.
+* `checkpointInterval`: If using checkpointing (set in the Spark configuration), this parameter specifies the frequency with which checkpoints will be created. If `maxIterations` is large, using checkpointing can help reduce shuffle file sizes on disk and help with failure recovery.
+
+*Note*: LDA is a new feature with some missing functionality. In particular, it does not yet
+support prediction on new documents, and it does not have a Python API. These will be added in the future.
+
+**Examples**
+
+In the following example, we load word count vectors representing a corpus of documents.
+We then use [LDA](api/scala/index.html#org.apache.spark.mllib.clustering.LDA)
+to infer three topics from the documents. The number of desired clusters is passed
+to the algorithm. We then output the topics, represented as probability distributions over words.
+
+
+
+
+{% highlight scala %}
+import org.apache.spark.mllib.clustering.LDA
+import org.apache.spark.mllib.linalg.Vectors
+
+// Load and parse the data
+val data = sc.textFile("data/mllib/sample_lda_data.txt")
+val parsedData = data.map(s => Vectors.dense(s.trim.split(' ').map(_.toDouble)))
+// Index documents with unique IDs
+val corpus = parsedData.zipWithIndex.map(_.swap).cache()
+
+// Cluster the documents into three topics using LDA
+val ldaModel = new LDA().setK(3).run(corpus)
+
+// Output topics. Each is a distribution over words (matching word count vectors)
+println("Learned topics (as distributions over vocab of " + ldaModel.vocabSize + " words):")
+val topics = ldaModel.topicsMatrix
+for (topic <- Range(0, 3)) {
+ print("Topic " + topic + ":")
+ for (word <- Range(0, ldaModel.vocabSize)) { print(" " + topics(word, topic)); }
+ println()
+}
+{% endhighlight %}
+
+
+
+{% highlight java %}
+import scala.Tuple2;
+
+import org.apache.spark.api.java.*;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.mllib.clustering.DistributedLDAModel;
+import org.apache.spark.mllib.clustering.LDA;
+import org.apache.spark.mllib.linalg.Matrix;
+import org.apache.spark.mllib.linalg.Vector;
+import org.apache.spark.mllib.linalg.Vectors;
+import org.apache.spark.SparkConf;
+
+public class JavaLDAExample {
+ public static void main(String[] args) {
+ SparkConf conf = new SparkConf().setAppName("LDA Example");
+ JavaSparkContext sc = new JavaSparkContext(conf);
+
+ // Load and parse the data
+ String path = "data/mllib/sample_lda_data.txt";
+ JavaRDD data = sc.textFile(path);
+ JavaRDD parsedData = data.map(
+ new Function() {
+ public Vector call(String s) {
+ String[] sarray = s.trim().split(" ");
+ double[] values = new double[sarray.length];
+ for (int i = 0; i < sarray.length; i++)
+ values[i] = Double.parseDouble(sarray[i]);
+ return Vectors.dense(values);
+ }
+ }
+ );
+ // Index documents with unique IDs
+ JavaPairRDD corpus = JavaPairRDD.fromJavaRDD(parsedData.zipWithIndex().map(
+ new Function, Tuple2>() {
+ public Tuple2 call(Tuple2 doc_id) {
+ return doc_id.swap();
+ }
+ }
+ ));
+ corpus.cache();
+
+ // Cluster the documents into three topics using LDA
+ DistributedLDAModel ldaModel = new LDA().setK(3).run(corpus);
+
+ // Output topics. Each is a distribution over words (matching word count vectors)
+ System.out.println("Learned topics (as distributions over vocab of " + ldaModel.vocabSize()
+ + " words):");
+ Matrix topics = ldaModel.topicsMatrix();
+ for (int topic = 0; topic < 3; topic++) {
+ System.out.print("Topic " + topic + ":");
+ for (int word = 0; word < ldaModel.vocabSize(); word++) {
+ System.out.print(" " + topics.apply(word, topic));
+ }
+ System.out.println();
+ }
+ }
+}
+{% endhighlight %}
+
+
+
+
+## Streaming k-means
+
+When data arrive in a stream, we may want to estimate clusters dynamically,
+updating them as new data arrive. MLlib provides support for streaming k-means clustering,
+with parameters to control the decay (or "forgetfulness") of the estimates. The algorithm
+uses a generalization of the mini-batch k-means update rule. For each batch of data, we assign
all points to their nearest cluster, compute new cluster centers, then update each cluster using:
`\begin{equation}
c_{t+1} = \frac{c_tn_t\alpha + x_tm_t}{n_t\alpha+m_t}
\end{equation}`
`\begin{equation}
- n_{t+1} = n_t + m_t
+ n_{t+1} = n_t + m_t
\end{equation}`
-Where `$c_t$` is the previous center for the cluster, `$n_t$` is the number of points assigned
-to the cluster thus far, `$x_t$` is the new cluster center from the current batch, and `$m_t$`
-is the number of points added to the cluster in the current batch. The decay factor `$\alpha$`
-can be used to ignore the past: with `$\alpha$=1` all data will be used from the beginning;
-with `$\alpha$=0` only the most recent data will be used. This is analogous to an
-exponentially-weighted moving average.
+Where `$c_t$` is the previous center for the cluster, `$n_t$` is the number of points assigned
+to the cluster thus far, `$x_t$` is the new cluster center from the current batch, and `$m_t$`
+is the number of points added to the cluster in the current batch. The decay factor `$\alpha$`
+can be used to ignore the past: with `$\alpha$=1` all data will be used from the beginning;
+with `$\alpha$=0` only the most recent data will be used. This is analogous to an
+exponentially-weighted moving average.
-The decay can be specified using a `halfLife` parameter, which determines the
+The decay can be specified using a `halfLife` parameter, which determines the
correct decay factor `a` such that, for data acquired
at time `t`, its contribution by time `t + halfLife` will have dropped to 0.5.
The unit of time can be specified either as `batches` or `points` and the update rule
will be adjusted accordingly.
-### Examples
+**Examples**
This example shows how to estimate clusters on streaming data.
@@ -220,9 +529,9 @@ import org.apache.spark.mllib.clustering.StreamingKMeans
{% endhighlight %}
-Then we make an input stream of vectors for training, as well as a stream of labeled data
-points for testing. We assume a StreamingContext `ssc` has been created, see
-[Spark Streaming Programming Guide](streaming-programming-guide.html#initializing) for more info.
+Then we make an input stream of vectors for training, as well as a stream of labeled data
+points for testing. We assume a StreamingContext `ssc` has been created, see
+[Spark Streaming Programming Guide](streaming-programming-guide.html#initializing) for more info.
{% highlight scala %}
@@ -244,24 +553,24 @@ val model = new StreamingKMeans()
{% endhighlight %}
-Now register the streams for training and testing and start the job, printing
+Now register the streams for training and testing and start the job, printing
the predicted cluster assignments on new data points as they arrive.
{% highlight scala %}
model.trainOn(trainingData)
-model.predictOnValues(testData).print()
+model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print()
ssc.start()
ssc.awaitTermination()
-
+
{% endhighlight %}
-As you add new text files with data the cluster centers will update. Each training
+As you add new text files with data the cluster centers will update. Each training
point should be formatted as `[x1, x2, x3]`, and each test data point
-should be formatted as `(y, [x1, x2, x3])`, where `y` is some useful label or identifier
-(e.g. a true category assignment). Anytime a text file is placed in `/training/data/dir`
-the model will update. Anytime a text file is placed in `/testing/data/dir`
+should be formatted as `(y, [x1, x2, x3])`, where `y` is some useful label or identifier
+(e.g. a true category assignment). Anytime a text file is placed in `/training/data/dir`
+the model will update. Anytime a text file is placed in `/testing/data/dir`
you will see predictions. With new data, the cluster centers will change!
diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md
index ef18cec9371d6..76140282a2dd0 100644
--- a/docs/mllib-collaborative-filtering.md
+++ b/docs/mllib-collaborative-filtering.md
@@ -66,6 +66,7 @@ recommendation model by measuring the Mean Squared Error of rating prediction.
{% highlight scala %}
import org.apache.spark.mllib.recommendation.ALS
+import org.apache.spark.mllib.recommendation.MatrixFactorizationModel
import org.apache.spark.mllib.recommendation.Rating
// Load and parse the data
@@ -95,6 +96,10 @@ val MSE = ratesAndPreds.map { case ((user, product), (r1, r2)) =>
err * err
}.mean()
println("Mean Squared Error = " + MSE)
+
+// Save and load model
+model.save(sc, "myModelPath")
+val sameModel = MatrixFactorizationModel.load(sc, "myModelPath")
{% endhighlight %}
If the rating matrix is derived from another source of information (e.g., it is inferred from
@@ -181,6 +186,10 @@ public class CollaborativeFiltering {
}
).rdd()).mean();
System.out.println("Mean Squared Error = " + MSE);
+
+ // Save and load model
+ model.save(sc.sc(), "myModelPath");
+ MatrixFactorizationModel sameModel = MatrixFactorizationModel.load(sc.sc(), "myModelPath");
}
}
{% endhighlight %}
@@ -192,7 +201,7 @@ We use the default ALS.train() method which assumes ratings are explicit. We eva
recommendation by measuring the Mean Squared Error of rating prediction.
{% highlight python %}
-from pyspark.mllib.recommendation import ALS, Rating
+from pyspark.mllib.recommendation import ALS, MatrixFactorizationModel, Rating
# Load and parse the data
data = sc.textFile("data/mllib/als/test.data")
@@ -209,6 +218,10 @@ predictions = model.predictAll(testdata).map(lambda r: ((r[0], r[1]), r[2]))
ratesAndPreds = ratings.map(lambda r: ((r[0], r[1]), r[2])).join(predictions)
MSE = ratesAndPreds.map(lambda r: (r[1][0] - r[1][1])**2).reduce(lambda x, y: x + y) / ratesAndPreds.count()
print("Mean Squared Error = " + str(MSE))
+
+# Save and load model
+model.save(sc, "myModelPath")
+sameModel = MatrixFactorizationModel.load(sc, "myModelPath")
{% endhighlight %}
If the rating matrix is derived from other source of information (i.e., it is inferred from other
diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md
index 101dc2f8695f3..fe6c1bf7bfd99 100644
--- a/docs/mllib-data-types.md
+++ b/docs/mllib-data-types.md
@@ -296,6 +296,70 @@ backed by an RDD of its entries.
The underlying RDDs of a distributed matrix must be deterministic, because we cache the matrix size.
In general the use of non-deterministic RDDs can lead to errors.
+### BlockMatrix
+
+A `BlockMatrix` is a distributed matrix backed by an RDD of `MatrixBlock`s, where a `MatrixBlock` is
+a tuple of `((Int, Int), Matrix)`, where the `(Int, Int)` is the index of the block, and `Matrix` is
+the sub-matrix at the given index with size `rowsPerBlock` x `colsPerBlock`.
+`BlockMatrix` supports methods such as `add` and `multiply` with another `BlockMatrix`.
+`BlockMatrix` also has a helper function `validate` which can be used to check whether the
+`BlockMatrix` is set up properly.
+
+
+
+
+A [`BlockMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.BlockMatrix) can be
+most easily created from an `IndexedRowMatrix` or `CoordinateMatrix` by calling `toBlockMatrix`.
+`toBlockMatrix` creates blocks of size 1024 x 1024 by default.
+Users may change the block size by supplying the values through `toBlockMatrix(rowsPerBlock, colsPerBlock)`.
+
+{% highlight scala %}
+import org.apache.spark.mllib.linalg.distributed.{BlockMatrix, CoordinateMatrix, MatrixEntry}
+
+val entries: RDD[MatrixEntry] = ... // an RDD of (i, j, v) matrix entries
+// Create a CoordinateMatrix from an RDD[MatrixEntry].
+val coordMat: CoordinateMatrix = new CoordinateMatrix(entries)
+// Transform the CoordinateMatrix to a BlockMatrix
+val matA: BlockMatrix = coordMat.toBlockMatrix().cache()
+
+// Validate whether the BlockMatrix is set up properly. Throws an Exception when it is not valid.
+// Nothing happens if it is valid.
+matA.validate()
+
+// Calculate A^T A.
+val ata = matA.transpose.multiply(matA)
+{% endhighlight %}
+
+
+
+
+A [`BlockMatrix`](api/java/org/apache/spark/mllib/linalg/distributed/BlockMatrix.html) can be
+most easily created from an `IndexedRowMatrix` or `CoordinateMatrix` by calling `toBlockMatrix`.
+`toBlockMatrix` creates blocks of size 1024 x 1024 by default.
+Users may change the block size by supplying the values through `toBlockMatrix(rowsPerBlock, colsPerBlock)`.
+
+{% highlight java %}
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.mllib.linalg.distributed.BlockMatrix;
+import org.apache.spark.mllib.linalg.distributed.CoordinateMatrix;
+import org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix;
+
+JavaRDD entries = ... // a JavaRDD of (i, j, v) Matrix Entries
+// Create a CoordinateMatrix from a JavaRDD.
+CoordinateMatrix coordMat = new CoordinateMatrix(entries.rdd());
+// Transform the CoordinateMatrix to a BlockMatrix
+BlockMatrix matA = coordMat.toBlockMatrix().cache();
+
+// Validate whether the BlockMatrix is set up properly. Throws an Exception when it is not valid.
+// Nothing happens if it is valid.
+matA.validate();
+
+// Calculate A^T A.
+BlockMatrix ata = matA.transpose().multiply(matA);
+{% endhighlight %}
+
+
+
### RowMatrix
A `RowMatrix` is a row-oriented distributed matrix without meaningful row indices, backed by an RDD
diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md
index fc8e732251a30..c1d0f8a6b1cd8 100644
--- a/docs/mllib-decision-tree.md
+++ b/docs/mllib-decision-tree.md
@@ -1,7 +1,7 @@
---
layout: global
-title: Decision Tree - MLlib
-displayTitle: MLlib - Decision Tree
+title: Decision Trees - MLlib
+displayTitle: MLlib - Decision Trees
---
* Table of contents
@@ -54,8 +54,8 @@ impurity measure for regression (variance).
Variance
Regression
- $\frac{1}{N} \sum_{i=1}^{N} (x_i - \mu)^2$ $y_i$ is label for an instance,
- $N$ is the number of instances and $\mu$ is the mean given by $\frac{1}{N} \sum_{i=1}^N x_i$.
+ $\frac{1}{N} \sum_{i=1}^{N} (y_i - \mu)^2$ $y_i$ is label for an instance,
+ $N$ is the number of instances and $\mu$ is the mean given by $\frac{1}{N} \sum_{i=1}^N y_i$.
@@ -194,6 +194,7 @@ maximum tree depth of 5. The test error is calculated to measure the algorithm a
{% highlight scala %}
import org.apache.spark.mllib.tree.DecisionTree
+import org.apache.spark.mllib.tree.model.DecisionTreeModel
import org.apache.spark.mllib.util.MLUtils
// Load and parse the data file.
@@ -221,6 +222,10 @@ val labelAndPreds = testData.map { point =>
val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count()
println("Test Error = " + testErr)
println("Learned classification tree model:\n" + model.toDebugString)
+
+// Save and load model
+model.save(sc, "myModelPath")
+val sameModel = DecisionTreeModel.load(sc, "myModelPath")
{% endhighlight %}
@@ -279,13 +284,18 @@ Double testErr =
}).count() / testData.count();
System.out.println("Test Error: " + testErr);
System.out.println("Learned classification tree model:\n" + model.toDebugString());
+
+// Save and load model
+model.save(sc.sc(), "myModelPath");
+DecisionTreeModel sameModel = DecisionTreeModel.load(sc.sc(), "myModelPath");
{% endhighlight %}
+
{% highlight python %}
from pyspark.mllib.regression import LabeledPoint
-from pyspark.mllib.tree import DecisionTree
+from pyspark.mllib.tree import DecisionTree, DecisionTreeModel
from pyspark.mllib.util import MLUtils
# Load and parse the data file into an RDD of LabeledPoint.
@@ -305,6 +315,10 @@ testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(tes
print('Test Error = ' + str(testErr))
print('Learned classification tree model:')
print(model.toDebugString())
+
+# Save and load model
+model.save(sc, "myModelPath")
+sameModel = DecisionTreeModel.load(sc, "myModelPath")
{% endhighlight %}
@@ -324,6 +338,7 @@ depth of 5. The Mean Squared Error (MSE) is computed at the end to evaluate
{% highlight scala %}
import org.apache.spark.mllib.tree.DecisionTree
+import org.apache.spark.mllib.tree.model.DecisionTreeModel
import org.apache.spark.mllib.util.MLUtils
// Load and parse the data file.
@@ -350,6 +365,10 @@ val labelsAndPredictions = testData.map { point =>
val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean()
println("Test Mean Squared Error = " + testMSE)
println("Learned regression tree model:\n" + model.toDebugString)
+
+// Save and load model
+model.save(sc, "myModelPath")
+val sameModel = DecisionTreeModel.load(sc, "myModelPath")
{% endhighlight %}
@@ -414,13 +433,18 @@ Double testMSE =
}) / data.count();
System.out.println("Test Mean Squared Error: " + testMSE);
System.out.println("Learned regression tree model:\n" + model.toDebugString());
+
+// Save and load model
+model.save(sc.sc(), "myModelPath");
+DecisionTreeModel sameModel = DecisionTreeModel.load(sc.sc(), "myModelPath");
{% endhighlight %}
+
{% highlight python %}
from pyspark.mllib.regression import LabeledPoint
-from pyspark.mllib.tree import DecisionTree
+from pyspark.mllib.tree import DecisionTree, DecisionTreeModel
from pyspark.mllib.util import MLUtils
# Load and parse the data file into an RDD of LabeledPoint.
@@ -440,6 +464,10 @@ testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / flo
print('Test Mean Squared Error = ' + str(testMSE))
print('Learned regression tree model:')
print(model.toDebugString())
+
+# Save and load model
+model.save(sc, "myModelPath")
+sameModel = DecisionTreeModel.load(sc, "myModelPath")
{% endhighlight %}
diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md
index 23ede04b62d5b..cbfb682609af3 100644
--- a/docs/mllib-ensembles.md
+++ b/docs/mllib-ensembles.md
@@ -98,6 +98,7 @@ The test error is calculated to measure the algorithm accuracy.
{% highlight scala %}
import org.apache.spark.mllib.tree.RandomForest
+import org.apache.spark.mllib.tree.model.RandomForestModel
import org.apache.spark.mllib.util.MLUtils
// Load and parse the data file.
@@ -127,6 +128,10 @@ val labelAndPreds = testData.map { point =>
val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count()
println("Test Error = " + testErr)
println("Learned classification forest model:\n" + model.toDebugString)
+
+// Save and load model
+model.save(sc, "myModelPath")
+val sameModel = RandomForestModel.load(sc, "myModelPath")
{% endhighlight %}
@@ -188,12 +193,17 @@ Double testErr =
}).count() / testData.count();
System.out.println("Test Error: " + testErr);
System.out.println("Learned classification forest model:\n" + model.toDebugString());
+
+// Save and load model
+model.save(sc.sc(), "myModelPath");
+RandomForestModel sameModel = RandomForestModel.load(sc.sc(), "myModelPath");
{% endhighlight %}
+
{% highlight python %}
-from pyspark.mllib.tree import RandomForest
+from pyspark.mllib.tree import RandomForest, RandomForestModel
from pyspark.mllib.util import MLUtils
# Load and parse the data file into an RDD of LabeledPoint.
@@ -216,6 +226,10 @@ testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(tes
print('Test Error = ' + str(testErr))
print('Learned classification forest model:')
print(model.toDebugString())
+
+# Save and load model
+model.save(sc, "myModelPath")
+sameModel = RandomForestModel.load(sc, "myModelPath")
{% endhighlight %}
@@ -235,6 +249,7 @@ The Mean Squared Error (MSE) is computed at the end to evaluate
{% highlight scala %}
import org.apache.spark.mllib.tree.RandomForest
+import org.apache.spark.mllib.tree.model.RandomForestModel
import org.apache.spark.mllib.util.MLUtils
// Load and parse the data file.
@@ -264,6 +279,10 @@ val labelsAndPredictions = testData.map { point =>
val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean()
println("Test Mean Squared Error = " + testMSE)
println("Learned regression forest model:\n" + model.toDebugString)
+
+// Save and load model
+model.save(sc, "myModelPath")
+val sameModel = RandomForestModel.load(sc, "myModelPath")
{% endhighlight %}
@@ -328,12 +347,17 @@ Double testMSE =
}) / testData.count();
System.out.println("Test Mean Squared Error: " + testMSE);
System.out.println("Learned regression forest model:\n" + model.toDebugString());
+
+// Save and load model
+model.save(sc.sc(), "myModelPath");
+RandomForestModel sameModel = RandomForestModel.load(sc.sc(), "myModelPath");
{% endhighlight %}
+
{% highlight python %}
-from pyspark.mllib.tree import RandomForest
+from pyspark.mllib.tree import RandomForest, RandomForestModel
from pyspark.mllib.util import MLUtils
# Load and parse the data file into an RDD of LabeledPoint.
@@ -356,6 +380,10 @@ testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / flo
print('Test Mean Squared Error = ' + str(testMSE))
print('Learned regression forest model:')
print(model.toDebugString())
+
+# Save and load model
+model.save(sc, "myModelPath")
+sameModel = RandomForestModel.load(sc, "myModelPath")
{% endhighlight %}
@@ -427,10 +455,19 @@ We omit some decision tree parameters since those are covered in the [decision t
* **`algo`**: The algorithm or task (classification vs. regression) is set using the tree [Strategy] parameter.
+#### Validation while training
-### Examples
+Gradient boosting can overfit when trained with more trees. In order to prevent overfitting, it is useful to validate while
+training. The method runWithValidation has been provided to make use of this option. It takes a pair of RDD's as arguments, the
+first one being the training dataset and the second being the validation dataset.
-GBTs currently have APIs in Scala and Java. Examples in both languages are shown below.
+The training is stopped when the improvement in the validation error is not more than a certain tolerance
+(supplied by the `validationTol` argument in `BoostingStrategy`). In practice, the validation error
+decreases initially and later increases. There might be cases in which the validation error does not change monotonically,
+and the user is advised to set a large enough negative tolerance and examine the validation curve to to tune the number of
+iterations.
+
+### Examples
#### Classification
@@ -446,6 +483,7 @@ The test error is calculated to measure the algorithm accuracy.
{% highlight scala %}
import org.apache.spark.mllib.tree.GradientBoostedTrees
import org.apache.spark.mllib.tree.configuration.BoostingStrategy
+import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel
import org.apache.spark.mllib.util.MLUtils
// Load and parse the data file.
@@ -458,7 +496,7 @@ val (trainingData, testData) = (splits(0), splits(1))
// The defaultParams for Classification use LogLoss by default.
val boostingStrategy = BoostingStrategy.defaultParams("Classification")
boostingStrategy.numIterations = 3 // Note: Use more iterations in practice.
-boostingStrategy.treeStrategy.numClassesForClassification = 2
+boostingStrategy.treeStrategy.numClasses = 2
boostingStrategy.treeStrategy.maxDepth = 5
// Empty categoricalFeaturesInfo indicates all features are continuous.
boostingStrategy.treeStrategy.categoricalFeaturesInfo = Map[Int, Int]()
@@ -473,6 +511,10 @@ val labelAndPreds = testData.map { point =>
val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count()
println("Test Error = " + testErr)
println("Learned classification GBT model:\n" + model.toDebugString)
+
+// Save and load model
+model.save(sc, "myModelPath")
+val sameModel = GradientBoostedTreesModel.load(sc, "myModelPath")
{% endhighlight %}
@@ -534,6 +576,41 @@ Double testErr =
}).count() / testData.count();
System.out.println("Test Error: " + testErr);
System.out.println("Learned classification GBT model:\n" + model.toDebugString());
+
+// Save and load model
+model.save(sc.sc(), "myModelPath");
+GradientBoostedTreesModel sameModel = GradientBoostedTreesModel.load(sc.sc(), "myModelPath");
+{% endhighlight %}
+
+
+
+
+{% highlight python %}
+from pyspark.mllib.tree import GradientBoostedTrees, GradientBoostedTreesModel
+from pyspark.mllib.util import MLUtils
+
+# Load and parse the data file.
+data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt")
+# Split the data into training and test sets (30% held out for testing)
+(trainingData, testData) = data.randomSplit([0.7, 0.3])
+
+# Train a GradientBoostedTrees model.
+# Notes: (a) Empty categoricalFeaturesInfo indicates all features are continuous.
+# (b) Use more iterations in practice.
+model = GradientBoostedTrees.trainClassifier(trainingData,
+ categoricalFeaturesInfo={}, numIterations=3)
+
+# Evaluate model on test instances and compute test error
+predictions = model.predict(testData.map(lambda x: x.features))
+labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions)
+testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(testData.count())
+print('Test Error = ' + str(testErr))
+print('Learned classification GBT model:')
+print(model.toDebugString())
+
+# Save and load model
+model.save(sc, "myModelPath")
+sameModel = GradientBoostedTreesModel.load(sc, "myModelPath")
{% endhighlight %}
@@ -554,6 +631,7 @@ The Mean Squared Error (MSE) is computed at the end to evaluate
{% highlight scala %}
import org.apache.spark.mllib.tree.GradientBoostedTrees
import org.apache.spark.mllib.tree.configuration.BoostingStrategy
+import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel
import org.apache.spark.mllib.util.MLUtils
// Load and parse the data file.
@@ -580,6 +658,10 @@ val labelsAndPredictions = testData.map { point =>
val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean()
println("Test Mean Squared Error = " + testMSE)
println("Learned regression GBT model:\n" + model.toDebugString)
+
+// Save and load model
+model.save(sc, "myModelPath")
+val sameModel = GradientBoostedTreesModel.load(sc, "myModelPath")
{% endhighlight %}
@@ -647,6 +729,41 @@ Double testMSE =
}) / data.count();
System.out.println("Test Mean Squared Error: " + testMSE);
System.out.println("Learned regression GBT model:\n" + model.toDebugString());
+
+// Save and load model
+model.save(sc.sc(), "myModelPath");
+GradientBoostedTreesModel sameModel = GradientBoostedTreesModel.load(sc.sc(), "myModelPath");
+{% endhighlight %}
+
+
+
+
+{% highlight python %}
+from pyspark.mllib.tree import GradientBoostedTrees, GradientBoostedTreesModel
+from pyspark.mllib.util import MLUtils
+
+# Load and parse the data file.
+data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt")
+# Split the data into training and test sets (30% held out for testing)
+(trainingData, testData) = data.randomSplit([0.7, 0.3])
+
+# Train a GradientBoostedTrees model.
+# Notes: (a) Empty categoricalFeaturesInfo indicates all features are continuous.
+# (b) Use more iterations in practice.
+model = GradientBoostedTrees.trainRegressor(trainingData,
+ categoricalFeaturesInfo={}, numIterations=3)
+
+# Evaluate model on test instances and compute test error
+predictions = model.predict(testData.map(lambda x: x.features))
+labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions)
+testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / float(testData.count())
+print('Test Mean Squared Error = ' + str(testMSE))
+print('Learned regression GBT model:')
+print(model.toDebugString())
+
+# Save and load model
+model.save(sc, "myModelPath")
+sameModel = GradientBoostedTreesModel.load(sc, "myModelPath")
{% endhighlight %}
diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md
index d4a61a7fbf3d7..80842b27effd8 100644
--- a/docs/mllib-feature-extraction.md
+++ b/docs/mllib-feature-extraction.md
@@ -375,3 +375,105 @@ data2 = labels.zip(normalizer2.transform(features))
{% endhighlight %}
+
+## Feature selection
+[Feature selection](http://en.wikipedia.org/wiki/Feature_selection) allows selecting the most relevant features for use in model construction. Feature selection reduces the size of the vector space and, in turn, the complexity of any subsequent operation with vectors. The number of features to select can be tuned using a held-out validation set.
+
+### ChiSqSelector
+[`ChiSqSelector`](api/scala/index.html#org.apache.spark.mllib.feature.ChiSqSelector) stands for Chi-Squared feature selection. It operates on labeled data with categorical features. `ChiSqSelector` orders features based on a Chi-Squared test of independence from the class, and then filters (selects) the top features which are most closely related to the label.
+
+#### Model Fitting
+
+[`ChiSqSelector`](api/scala/index.html#org.apache.spark.mllib.feature.ChiSqSelector) has the
+following parameters in the constructor:
+
+* `numTopFeatures` number of top features that the selector will select (filter).
+
+We provide a [`fit`](api/scala/index.html#org.apache.spark.mllib.feature.ChiSqSelector) method in
+`ChiSqSelector` which can take an input of `RDD[LabeledPoint]` with categorical features, learn the summary statistics, and then
+return a `ChiSqSelectorModel` which can transform an input dataset into the reduced feature space.
+
+This model implements [`VectorTransformer`](api/scala/index.html#org.apache.spark.mllib.feature.VectorTransformer)
+which can apply the Chi-Squared feature selection on a `Vector` to produce a reduced `Vector` or on
+an `RDD[Vector]` to produce a reduced `RDD[Vector]`.
+
+Note that the user can also construct a `ChiSqSelectorModel` by hand by providing an array of selected feature indices (which must be sorted in ascending order).
+
+#### Example
+
+The following example shows the basic use of ChiSqSelector.
+
+
+
+{% highlight scala %}
+import org.apache.spark.SparkContext._
+import org.apache.spark.mllib.linalg.Vectors
+import org.apache.spark.mllib.regression.LabeledPoint
+import org.apache.spark.mllib.util.MLUtils
+
+// Load some data in libsvm format
+val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt")
+// Discretize data in 16 equal bins since ChiSqSelector requires categorical features
+val discretizedData = data.map { lp =>
+ LabeledPoint(lp.label, Vectors.dense(lp.features.toArray.map { x => x / 16 } ) )
+}
+// Create ChiSqSelector that will select 50 features
+val selector = new ChiSqSelector(50)
+// Create ChiSqSelector model (selecting features)
+val transformer = selector.fit(discretizedData)
+// Filter the top 50 features from each feature vector
+val filteredData = discretizedData.map { lp =>
+ LabeledPoint(lp.label, transformer.transform(lp.features))
+}
+{% endhighlight %}
+
+
+
+{% highlight java %}
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.mllib.feature.ChiSqSelector;
+import org.apache.spark.mllib.feature.ChiSqSelectorModel;
+import org.apache.spark.mllib.linalg.Vectors;
+import org.apache.spark.mllib.regression.LabeledPoint;
+import org.apache.spark.mllib.util.MLUtils;
+
+SparkConf sparkConf = new SparkConf().setAppName("JavaChiSqSelector");
+JavaSparkContext sc = new JavaSparkContext(sparkConf);
+JavaRDD points = MLUtils.loadLibSVMFile(sc.sc(),
+ "data/mllib/sample_libsvm_data.txt").toJavaRDD().cache();
+
+// Discretize data in 16 equal bins since ChiSqSelector requires categorical features
+JavaRDD discretizedData = points.map(
+ new Function() {
+ @Override
+ public LabeledPoint call(LabeledPoint lp) {
+ final double[] discretizedFeatures = new double[lp.features().size()];
+ for (int i = 0; i < lp.features().size(); ++i) {
+ discretizedFeatures[i] = lp.features().apply(i) / 16;
+ }
+ return new LabeledPoint(lp.label(), Vectors.dense(discretizedFeatures));
+ }
+ });
+
+// Create ChiSqSelector that will select 50 features
+ChiSqSelector selector = new ChiSqSelector(50);
+// Create ChiSqSelector model (selecting features)
+final ChiSqSelectorModel transformer = selector.fit(discretizedData.rdd());
+// Filter the top 50 features from each feature vector
+JavaRDD filteredData = discretizedData.map(
+ new Function() {
+ @Override
+ public LabeledPoint call(LabeledPoint lp) {
+ return new LabeledPoint(lp.label(), transformer.transform(lp.features()));
+ }
+ }
+);
+
+sc.stop();
+{% endhighlight %}
+
+
+
diff --git a/docs/mllib-frequent-pattern-mining.md b/docs/mllib-frequent-pattern-mining.md
new file mode 100644
index 0000000000000..9fd9be0dd01b1
--- /dev/null
+++ b/docs/mllib-frequent-pattern-mining.md
@@ -0,0 +1,98 @@
+---
+layout: global
+title: Frequent Pattern Mining - MLlib
+displayTitle: MLlib - Frequent Pattern Mining
+---
+
+Mining frequent items, itemsets, subsequences, or other substructures is usually among the
+first steps to analyze a large-scale dataset, which has been an active research topic in
+data mining for years.
+We refer users to Wikipedia's [association rule learning](http://en.wikipedia.org/wiki/Association_rule_learning)
+for more information.
+MLlib provides a parallel implementation of FP-growth,
+a popular algorithm to mining frequent itemsets.
+
+## FP-growth
+
+The FP-growth algorithm is described in the paper
+[Han et al., Mining frequent patterns without candidate generation](http://dx.doi.org/10.1145/335191.335372),
+where "FP" stands for frequent pattern.
+Given a dataset of transactions, the first step of FP-growth is to calculate item frequencies and identify frequent items.
+Different from [Apriori-like](http://en.wikipedia.org/wiki/Apriori_algorithm) algorithms designed for the same purpose,
+the second step of FP-growth uses a suffix tree (FP-tree) structure to encode transactions without generating candidate sets
+explicitly, which are usually expensive to generate.
+After the second step, the frequent itemsets can be extracted from the FP-tree.
+In MLlib, we implemented a parallel version of FP-growth called PFP,
+as described in [Li et al., PFP: Parallel FP-growth for query recommendation](http://dx.doi.org/10.1145/1454008.1454027).
+PFP distributes the work of growing FP-trees based on the suffices of transactions,
+and hence more scalable than a single-machine implementation.
+We refer users to the papers for more details.
+
+MLlib's FP-growth implementation takes the following (hyper-)parameters:
+
+* `minSupport`: the minimum support for an itemset to be identified as frequent.
+ For example, if an item appears 3 out of 5 transactions, it has a support of 3/5=0.6.
+* `numPartitions`: the number of partitions used to distribute the work.
+
+**Examples**
+
+
+
+
+[`FPGrowth`](api/java/org/apache/spark/mllib/fpm/FPGrowth.html) implements the
+FP-growth algorithm.
+It take a `JavaRDD` of transactions, where each transaction is an `Iterable` of items of a generic type.
+Calling `FPGrowth.run` with transactions returns an
+[`FPGrowthModel`](api/java/org/apache/spark/mllib/fpm/FPGrowthModel.html)
+that stores the frequent itemsets with their frequencies.
+
+{% highlight scala %}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.mllib.fpm.{FPGrowth, FPGrowthModel}
+
+val transactions: RDD[Array[String]] = ...
+
+val fpg = new FPGrowth()
+ .setMinSupport(0.2)
+ .setNumPartitions(10)
+val model = fpg.run(transactions)
+
+model.freqItemsets.collect().foreach { itemset =>
+ println(itemset.items.mkString("[", ",", "]") + ", " + itemset.freq)
+}
+{% endhighlight %}
+
+
+
+
+
+[`FPGrowth`](api/java/org/apache/spark/mllib/fpm/FPGrowth.html) implements the
+FP-growth algorithm.
+It take an `RDD` of transactions, where each transaction is an `Array` of items of a generic type.
+Calling `FPGrowth.run` with transactions returns an
+[`FPGrowthModel`](api/java/org/apache/spark/mllib/fpm/FPGrowthModel.html)
+that stores the frequent itemsets with their frequencies.
+
+{% highlight java %}
+import java.util.List;
+
+import com.google.common.base.Joiner;
+
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.mllib.fpm.FPGrowth;
+import org.apache.spark.mllib.fpm.FPGrowthModel;
+
+JavaRDD> transactions = ...
+
+FPGrowth fpg = new FPGrowth()
+ .setMinSupport(0.2)
+ .setNumPartitions(10);
+FPGrowthModel model = fpg.run(transactions);
+
+for (FPGrowth.FreqItemset itemset: model.freqItemsets().toJavaRDD().collect()) {
+ System.out.println("[" + Joiner.on(",").join(s.javaItems()) + "], " + s.freq());
+}
+{% endhighlight %}
+
+
+
diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md
index 39c64d06926bf..f8e879496c135 100644
--- a/docs/mllib-guide.md
+++ b/docs/mllib-guide.md
@@ -1,6 +1,8 @@
---
layout: global
-title: Machine Learning Library (MLlib) Programming Guide
+title: MLlib
+displayTitle: Machine Learning Library (MLlib) Guide
+description: MLlib machine learning library overview for Spark SPARK_VERSION_SHORT
---
MLlib is Spark's scalable machine learning library consisting of common learning algorithms and utilities,
@@ -19,14 +21,21 @@ filtering, dimensionality reduction, as well as underlying optimization primitiv
* [naive Bayes](mllib-naive-bayes.html)
* [decision trees](mllib-decision-tree.html)
* [ensembles of trees](mllib-ensembles.html) (Random Forests and Gradient-Boosted Trees)
+ * [isotonic regression](mllib-isotonic-regression.html)
* [Collaborative filtering](mllib-collaborative-filtering.html)
* alternating least squares (ALS)
* [Clustering](mllib-clustering.html)
- * k-means
+ * [k-means](mllib-clustering.html#k-means)
+ * [Gaussian mixture](mllib-clustering.html#gaussian-mixture)
+ * [power iteration clustering (PIC)](mllib-clustering.html#power-iteration-clustering-pic)
+ * [latent Dirichlet allocation (LDA)](mllib-clustering.html#latent-dirichlet-allocation-lda)
+ * [streaming k-means](mllib-clustering.html#streaming-k-means)
* [Dimensionality reduction](mllib-dimensionality-reduction.html)
* singular value decomposition (SVD)
* principal component analysis (PCA)
* [Feature extraction and transformation](mllib-feature-extraction.html)
+* [Frequent pattern mining](mllib-frequent-pattern-mining.html)
+ * FP-growth
* [Optimization (developer)](mllib-optimization.html)
* stochastic gradient descent
* limited-memory BFGS (L-BFGS)
@@ -37,7 +46,7 @@ and the migration guide below will explain all changes between releases.
# spark.ml: high-level APIs for ML pipelines
-Spark 1.2 includes a new package called `spark.ml`, which aims to provide a uniform set of
+Spark 1.2 introduced a new package called `spark.ml`, which aims to provide a uniform set of
high-level APIs that help users create and tune practical machine learning pipelines.
It is currently an alpha component, and we would like to hear back from the community about
how it fits real-world use cases and how it could be improved.
@@ -52,149 +61,50 @@ See the **[spark.ml programming guide](ml-guide.html)** for more information on
# Dependencies
-MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/),
-which depends on [netlib-java](https://github.com/fommil/netlib-java),
-and [jblas](https://github.com/mikiobraun/jblas).
-`netlib-java` and `jblas` depend on native Fortran routines.
-You need to install the
-[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries)
-if it is not already present on your nodes.
-MLlib will throw a linking error if it cannot detect these libraries automatically.
-Due to license issues, we do not include `netlib-java`'s native libraries in MLlib's
-dependency set under default settings.
-If no native library is available at runtime, you will see a warning message.
-To use native libraries from `netlib-java`, please build Spark with `-Pnetlib-lgpl` or
-include `com.github.fommil.netlib:all:1.1.2` as a dependency of your project.
-If you want to use optimized BLAS/LAPACK libraries such as
-[OpenBLAS](http://www.openblas.net/), please link its shared libraries to
-`/usr/lib/libblas.so.3` and `/usr/lib/liblapack.so.3`, respectively.
-BLAS/LAPACK libraries on worker nodes should be built without multithreading.
-
-To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer.
+MLlib uses the linear algebra package
+[Breeze](http://www.scalanlp.org/), which depends on
+[netlib-java](https://github.com/fommil/netlib-java) for optimised
+numerical processing. If natives are not available at runtime, you
+will see a warning message and a pure JVM implementation will be used
+instead.
----
-
-# Migration Guide
-
-## From 1.1 to 1.2
-
-The only API changes in MLlib v1.2 are in
-[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree),
-which continues to be an experimental API in MLlib 1.2:
-
-1. *(Breaking change)* The Scala API for classification takes a named argument specifying the number
-of classes. In MLlib v1.1, this argument was called `numClasses` in Python and
-`numClassesForClassification` in Scala. In MLlib v1.2, the names are both set to `numClasses`.
-This `numClasses` parameter is specified either via
-[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy)
-or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree)
-static `trainClassifier` and `trainRegressor` methods.
-
-2. *(Breaking change)* The API for
-[`Node`](api/scala/index.html#org.apache.spark.mllib.tree.model.Node) has changed.
-This should generally not affect user code, unless the user manually constructs decision trees
-(instead of using the `trainClassifier` or `trainRegressor` methods).
-The tree `Node` now includes more information, including the probability of the predicted label
-(for classification).
-
-3. Printing methods' output has changed. The `toString` (Scala/Java) and `__repr__` (Python) methods used to print the full model; they now print a summary. For the full model, use `toDebugString`.
-
-Examples in the Spark distribution and examples in the
-[Decision Trees Guide](mllib-decision-tree.html#examples) have been updated accordingly.
-
-## From 1.0 to 1.1
-
-The only API changes in MLlib v1.1 are in
-[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree),
-which continues to be an experimental API in MLlib 1.1:
-
-1. *(Breaking change)* The meaning of tree depth has been changed by 1 in order to match
-the implementations of trees in
-[scikit-learn](http://scikit-learn.org/stable/modules/classes.html#module-sklearn.tree)
-and in [rpart](http://cran.r-project.org/web/packages/rpart/index.html).
-In MLlib v1.0, a depth-1 tree had 1 leaf node, and a depth-2 tree had 1 root node and 2 leaf nodes.
-In MLlib v1.1, a depth-0 tree has 1 leaf node, and a depth-1 tree has 1 root node and 2 leaf nodes.
-This depth is specified by the `maxDepth` parameter in
-[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy)
-or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree)
-static `trainClassifier` and `trainRegressor` methods.
-
-2. *(Non-breaking change)* We recommend using the newly added `trainClassifier` and `trainRegressor`
-methods to build a [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree),
-rather than using the old parameter class `Strategy`. These new training methods explicitly
-separate classification and regression, and they replace specialized parameter types with
-simple `String` types.
+To learn more about the benefits and background of system optimised
+natives, you may wish to watch Sam Halliday's ScalaX talk on
+[High Performance Linear Algebra in Scala](http://fommil.github.io/scalax14/#/)).
-Examples of the new, recommended `trainClassifier` and `trainRegressor` are given in the
-[Decision Trees Guide](mllib-decision-tree.html#examples).
+Due to licensing issues with runtime proprietary binaries, we do not
+include `netlib-java`'s native proxies by default. To configure
+`netlib-java` / Breeze to use system optimised binaries, include
+`com.github.fommil.netlib:all:1.1.2` (or build Spark with
+`-Pnetlib-lgpl`) as a dependency of your project and read the
+[netlib-java](https://github.com/fommil/netlib-java) documentation for
+your platform's additional installation instructions.
-## From 0.9 to 1.0
+To use MLlib in Python, you will need [NumPy](http://www.numpy.org)
+version 1.4 or newer.
-In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few
-breaking changes. If your data is sparse, please store it in a sparse format instead of dense to
-take advantage of sparsity in both storage and computation. Details are described below.
-
-
-
-
-We used to represent a feature vector by `Array[Double]`, which is replaced by
-[`Vector`](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) in v1.0. Algorithms that used
-to accept `RDD[Array[Double]]` now take
-`RDD[Vector]`. [`LabeledPoint`](api/scala/index.html#org.apache.spark.mllib.regression.LabeledPoint)
-is now a wrapper of `(Double, Vector)` instead of `(Double, Array[Double])`. Converting
-`Array[Double]` to `Vector` is straightforward:
-
-{% highlight scala %}
-import org.apache.spark.mllib.linalg.{Vector, Vectors}
-
-val array: Array[Double] = ... // a double array
-val vector: Vector = Vectors.dense(array) // a dense vector
-{% endhighlight %}
-
-[`Vectors`](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to create sparse vectors.
-
-*Note*: Scala imports `scala.collection.immutable.Vector` by default, so you have to import `org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`.
-
-
-
-
-
-We used to represent a feature vector by `double[]`, which is replaced by
-[`Vector`](api/java/index.html?org/apache/spark/mllib/linalg/Vector.html) in v1.0. Algorithms that used
-to accept `RDD` now take
-`RDD`. [`LabeledPoint`](api/java/index.html?org/apache/spark/mllib/regression/LabeledPoint.html)
-is now a wrapper of `(double, Vector)` instead of `(double, double[])`. Converting `double[]` to
-`Vector` is straightforward:
-
-{% highlight java %}
-import org.apache.spark.mllib.linalg.Vector;
-import org.apache.spark.mllib.linalg.Vectors;
+---
-double[] array = ... // a double array
-Vector vector = Vectors.dense(array); // a dense vector
-{% endhighlight %}
+# Migration Guide
-[`Vectors`](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to
-create sparse vectors.
+For the `spark.ml` package, please see the [spark.ml Migration Guide](ml-guide.html#migration-guide).
-
-
-
+## From 1.2 to 1.3
-We used to represent a labeled feature vector in a NumPy array, where the first entry corresponds to
-the label and the rest are features. This representation is replaced by class
-[`LabeledPoint`](api/python/pyspark.mllib.regression.LabeledPoint-class.html), which takes both
-dense and sparse feature vectors.
+In the `spark.mllib` package, there were several breaking changes. The first change (in `ALS`) is the only one in a component not marked as Alpha or Experimental.
-{% highlight python %}
-from pyspark.mllib.linalg import SparseVector
-from pyspark.mllib.regression import LabeledPoint
+* *(Breaking change)* In [`ALS`](api/scala/index.html#org.apache.spark.mllib.recommendation.ALS), the extraneous method `solveLeastSquares` has been removed. The `DeveloperApi` method `analyzeBlocks` was also removed.
+* *(Breaking change)* [`StandardScalerModel`](api/scala/index.html#org.apache.spark.mllib.feature.StandardScalerModel) remains an Alpha component. In it, the `variance` method has been replaced with the `std` method. To compute the column variance values returned by the original `variance` method, simply square the standard deviation values returned by `std`.
+* *(Breaking change)* [`StreamingLinearRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD) remains an Experimental component. In it, there were two changes:
+ * The constructor taking arguments was removed in favor of a builder patten using the default constructor plus parameter setter methods.
+ * Variable `model` is no longer public.
+* *(Breaking change)* [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) remains an Experimental component. In it and its associated classes, there were several changes:
+ * In `DecisionTree`, the deprecated class method `train` has been removed. (The object/static `train` methods remain.)
+ * In `Strategy`, the `checkpointDir` parameter has been removed. Checkpointing is still supported, but the checkpoint directory must be set before calling tree and tree ensemble training.
+* `PythonMLlibAPI` (the interface between Scala/Java and Python for MLlib) was a public API but is now private, declared `private[python]`. This was never meant for external use.
+* In linear regression (including Lasso and ridge regression), the squared loss is now divided by 2.
+ So in order to produce the same result as in 1.2, the regularization parameter needs to be divided by 2 and the step size needs to be multiplied by 2.
-# Create a labeled point with a positive label and a dense feature vector.
-pos = LabeledPoint(1.0, [1.0, 0.0, 3.0])
+## Previous Spark Versions
-# Create a labeled point with a negative label and a sparse feature vector.
-neg = LabeledPoint(0.0, SparseVector(3, [0, 2], [1.0, 3.0]))
-{% endhighlight %}
-
-
+Earlier migration guides are archived [on this page](mllib-migration-guides.html).
diff --git a/docs/mllib-isotonic-regression.md b/docs/mllib-isotonic-regression.md
new file mode 100644
index 0000000000000..12fb29d426741
--- /dev/null
+++ b/docs/mllib-isotonic-regression.md
@@ -0,0 +1,155 @@
+---
+layout: global
+title: Naive Bayes - MLlib
+displayTitle: MLlib - Regression
+---
+
+## Isotonic regression
+[Isotonic regression](http://en.wikipedia.org/wiki/Isotonic_regression)
+belongs to the family of regression algorithms. Formally isotonic regression is a problem where
+given a finite set of real numbers `$Y = {y_1, y_2, ..., y_n}$` representing observed responses
+and `$X = {x_1, x_2, ..., x_n}$` the unknown response values to be fitted
+finding a function that minimises
+
+`\begin{equation}
+ f(x) = \sum_{i=1}^n w_i (y_i - x_i)^2
+\end{equation}`
+
+with respect to complete order subject to
+`$x_1\le x_2\le ...\le x_n$` where `$w_i$` are positive weights.
+The resulting function is called isotonic regression and it is unique.
+It can be viewed as least squares problem under order restriction.
+Essentially isotonic regression is a
+[monotonic function](http://en.wikipedia.org/wiki/Monotonic_function)
+best fitting the original data points.
+
+MLlib supports a
+[pool adjacent violators algorithm](http://doi.org/10.1198/TECH.2010.10111)
+which uses an approach to
+[parallelizing isotonic regression](http://doi.org/10.1007/978-3-642-99789-1_10).
+The training input is a RDD of tuples of three double values that represent
+label, feature and weight in this order. Additionally IsotonicRegression algorithm has one
+optional parameter called $isotonic$ defaulting to true.
+This argument specifies if the isotonic regression is
+isotonic (monotonically increasing) or antitonic (monotonically decreasing).
+
+Training returns an IsotonicRegressionModel that can be used to predict
+labels for both known and unknown features. The result of isotonic regression
+is treated as piecewise linear function. The rules for prediction therefore are:
+
+* If the prediction input exactly matches a training feature
+ then associated prediction is returned. In case there are multiple predictions with the same
+ feature then one of them is returned. Which one is undefined
+ (same as java.util.Arrays.binarySearch).
+* If the prediction input is lower or higher than all training features
+ then prediction with lowest or highest feature is returned respectively.
+ In case there are multiple predictions with the same feature
+ then the lowest or highest is returned respectively.
+* If the prediction input falls between two training features then prediction is treated
+ as piecewise linear function and interpolated value is calculated from the
+ predictions of the two closest features. In case there are multiple values
+ with the same feature then the same rules as in previous point are used.
+
+### Examples
+
+
+
+Data are read from a file where each line has a format label,feature
+i.e. 4710.28,500.00. The data are split to training and testing set.
+Model is created using the training set and a mean squared error is calculated from the predicted
+labels and real labels in the test set.
+
+{% highlight scala %}
+import org.apache.spark.mllib.regression.IsotonicRegression
+
+val data = sc.textFile("data/mllib/sample_isotonic_regression_data.txt")
+
+// Create label, feature, weight tuples from input data with weight set to default value 1.0.
+val parsedData = data.map { line =>
+ val parts = line.split(',').map(_.toDouble)
+ (parts(0), parts(1), 1.0)
+}
+
+// Split data into training (60%) and test (40%) sets.
+val splits = parsedData.randomSplit(Array(0.6, 0.4), seed = 11L)
+val training = splits(0)
+val test = splits(1)
+
+// Create isotonic regression model from training data.
+// Isotonic parameter defaults to true so it is only shown for demonstration
+val model = new IsotonicRegression().setIsotonic(true).run(training)
+
+// Create tuples of predicted and real labels.
+val predictionAndLabel = test.map { point =>
+ val predictedLabel = model.predict(point._2)
+ (predictedLabel, point._1)
+}
+
+// Calculate mean squared error between predicted and real labels.
+val meanSquaredError = predictionAndLabel.map{case(p, l) => math.pow((p - l), 2)}.mean()
+println("Mean Squared Error = " + meanSquaredError)
+{% endhighlight %}
+
+
+
+Data are read from a file where each line has a format label,feature
+i.e. 4710.28,500.00. The data are split to training and testing set.
+Model is created using the training set and a mean squared error is calculated from the predicted
+labels and real labels in the test set.
+
+{% highlight java %}
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaDoubleRDD;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.mllib.regression.IsotonicRegressionModel;
+import scala.Tuple2;
+import scala.Tuple3;
+
+JavaRDD data = sc.textFile("data/mllib/sample_isotonic_regression_data.txt");
+
+// Create label, feature, weight tuples from input data with weight set to default value 1.0.
+JavaRDD> parsedData = data.map(
+ new Function>() {
+ public Tuple3 call(String line) {
+ String[] parts = line.split(",");
+ return new Tuple3<>(new Double(parts[0]), new Double(parts[1]), 1.0);
+ }
+ }
+);
+
+// Split data into training (60%) and test (40%) sets.
+JavaRDD>[] splits = parsedData.randomSplit(new double[] {0.6, 0.4}, 11L);
+JavaRDD> training = splits[0];
+JavaRDD> test = splits[1];
+
+// Create isotonic regression model from training data.
+// Isotonic parameter defaults to true so it is only shown for demonstration
+IsotonicRegressionModel model = new IsotonicRegression().setIsotonic(true).run(training);
+
+// Create tuples of predicted and real labels.
+JavaPairRDD predictionAndLabel = test.mapToPair(
+ new PairFunction, Double, Double>() {
+ @Override public Tuple2 call(Tuple3 point) {
+ Double predictedLabel = model.predict(point._2());
+ return new Tuple2(predictedLabel, point._1());
+ }
+ }
+);
+
+// Calculate mean squared error between predicted and real labels.
+Double meanSquaredError = new JavaDoubleRDD(predictionAndLabel.map(
+ new Function, Object>() {
+ @Override public Object call(Tuple2 pl) {
+ return Math.pow(pl._1() - pl._2(), 2);
+ }
+ }
+).rdd()).mean();
+
+System.out.println("Mean Squared Error = " + meanSquaredError);
+{% endhighlight %}
+
+
\ No newline at end of file
diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md
index 44b7f67c57734..9270741d439d9 100644
--- a/docs/mllib-linear-methods.md
+++ b/docs/mllib-linear-methods.md
@@ -17,7 +17,7 @@ displayTitle: MLlib - Linear Methods
\newcommand{\av}{\mathbf{\alpha}}
\newcommand{\bv}{\mathbf{b}}
\newcommand{\N}{\mathbb{N}}
-\newcommand{\id}{\mathbf{I}}
+\newcommand{\id}{\mathbf{I}}
\newcommand{\ind}{\mathbf{1}}
\newcommand{\0}{\mathbf{0}}
\newcommand{\unit}{\mathbf{e}}
@@ -114,18 +114,26 @@ especially when the number of training examples is small.
Under the hood, linear methods use convex optimization methods to optimize the objective functions. MLlib uses two methods, SGD and L-BFGS, described in the [optimization section](mllib-optimization.html). Currently, most algorithm APIs support Stochastic Gradient Descent (SGD), and a few support L-BFGS. Refer to [this optimization section](mllib-optimization.html#Choosing-an-Optimization-Method) for guidelines on choosing between optimization methods.
-## Binary classification
-
-[Binary classification](http://en.wikipedia.org/wiki/Binary_classification)
-aims to divide items into two categories: positive and negative. MLlib
-supports two linear methods for binary classification: linear Support Vector
-Machines (SVMs) and logistic regression. For both methods, MLlib supports
-L1 and L2 regularized variants. The training data set is represented by an RDD
-of [LabeledPoint](mllib-data-types.html) in MLlib. Note that, in the
-mathematical formulation in this guide, a training label $y$ is denoted as
-either $+1$ (positive) or $-1$ (negative), which is convenient for the
-formulation. *However*, the negative label is represented by $0$ in MLlib
-instead of $-1$, to be consistent with multiclass labeling.
+## Classification
+
+[Classification](http://en.wikipedia.org/wiki/Statistical_classification) aims to divide items into
+categories.
+The most common classification type is
+[binary classificaion](http://en.wikipedia.org/wiki/Binary_classification), where there are two
+categories, usually named positive and negative.
+If there are more than two categories, it is called
+[multiclass classification](http://en.wikipedia.org/wiki/Multiclass_classification).
+MLlib supports two linear methods for classification: linear Support Vector Machines (SVMs)
+and logistic regression.
+Linear SVMs supports only binary classification, while logistic regression supports both binary and
+multiclass classification problems.
+For both methods, MLlib supports L1 and L2 regularized variants.
+The training data set is represented by an RDD of [LabeledPoint](mllib-data-types.html) in MLlib,
+where labels are class indices starting from zero: $0, 1, 2, \ldots$.
+Note that, in the mathematical formulation in this guide, a binary label $y$ is denoted as either
+$+1$ (positive) or $-1$ (negative), which is convenient for the formulation.
+*However*, the negative label is represented by $0$ in MLlib instead of $-1$, to be consistent with
+multiclass labeling.
### Linear Support Vector Machines (SVMs)
@@ -144,41 +152,7 @@ denoted by $\x$, the model makes predictions based on the value of $\wv^T \x$.
By the default, if $\wv^T \x \geq 0$ then the outcome is positive, and negative
otherwise.
-### Logistic regression
-
-[Logistic regression](http://en.wikipedia.org/wiki/Logistic_regression) is widely used to predict a
-binary response.
-It is a linear method as described above in equation `$\eqref{eq:regPrimal}$`, with the loss
-function in the formulation given by the logistic loss:
-`\[
-L(\wv;\x,y) := \log(1+\exp( -y \wv^T \x)).
-\]`
-
-The logistic regression algorithm outputs a logistic regression model. Given a
-new data point, denoted by $\x$, the model makes predictions by
-applying the logistic function
-`\[
-\mathrm{f}(z) = \frac{1}{1 + e^{-z}}
-\]`
-where $z = \wv^T \x$.
-By default, if $\mathrm{f}(\wv^T x) > 0.5$, the outcome is positive, or
-negative otherwise, though unlike linear SVMs, the raw output of the logistic regression
-model, $\mathrm{f}(z)$, has a probabilistic interpretation (i.e., the probability
-that $\x$ is positive).
-
-### Evaluation metrics
-
-MLlib supports common evaluation metrics for binary classification (not available in PySpark).
-This
-includes precision, recall, [F-measure](http://en.wikipedia.org/wiki/F1_score),
-[receiver operating characteristic (ROC)](http://en.wikipedia.org/wiki/Receiver_operating_characteristic),
-precision-recall curve, and
-[area under the curves (AUC)](http://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve).
-AUC is commonly used to compare the performance of various models while
-precision/recall/F-measure can help determine the appropriate threshold to use
-for prediction purposes.
-
-### Examples
+**Examples**
@@ -190,7 +164,7 @@ error.
{% highlight scala %}
import org.apache.spark.SparkContext
-import org.apache.spark.mllib.classification.SVMWithSGD
+import org.apache.spark.mllib.classification.{SVMModel, SVMWithSGD}
import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics
import org.apache.spark.mllib.regression.LabeledPoint
import org.apache.spark.mllib.linalg.Vectors
@@ -211,7 +185,7 @@ val model = SVMWithSGD.train(training, numIterations)
// Clear the default threshold.
model.clearThreshold()
-// Compute raw scores on the test set.
+// Compute raw scores on the test set.
val scoreAndLabels = test.map { point =>
val score = model.predict(point.features)
(score, point.label)
@@ -222,6 +196,10 @@ val metrics = new BinaryClassificationMetrics(scoreAndLabels)
val auROC = metrics.areaUnderROC()
println("Area under ROC = " + auROC)
+
+// Save and load model
+model.save(sc, "myModelPath")
+val sameModel = SVMModel.load(sc, "myModelPath")
{% endhighlight %}
The `SVMWithSGD.train()` method by default performs L2 regularization with the
@@ -243,8 +221,6 @@ svmAlg.optimizer.
val modelL1 = svmAlg.run(training)
{% endhighlight %}
-[`LogisticRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) can be used in a similar fashion as `SVMWithSGD`.
-
@@ -280,11 +256,11 @@ public class SVMClassifier {
JavaRDD training = data.sample(false, 0.6, 11L);
training.cache();
JavaRDD test = data.subtract(training);
-
+
// Run training algorithm to build the model.
int numIterations = 100;
final SVMModel model = SVMWithSGD.train(training.rdd(), numIterations);
-
+
// Clear the default threshold.
model.clearThreshold();
@@ -297,13 +273,17 @@ public class SVMClassifier {
}
}
);
-
+
// Get evaluation metrics.
- BinaryClassificationMetrics metrics =
+ BinaryClassificationMetrics metrics =
new BinaryClassificationMetrics(JavaRDD.toRDD(scoreAndLabels));
double auROC = metrics.areaUnderROC();
-
+
System.out.println("Area under ROC = " + auROC);
+
+ // Save and load model
+ model.save(sc.sc(), "myModelPath");
+ SVMModel sameModel = SVMModel.load(sc.sc(), "myModelPath");
}
}
{% endhighlight %}
@@ -338,6 +318,8 @@ a dependency.
The following example shows how to load a sample dataset, build Logistic Regression model,
and make predictions with the resulting model to compute the training error.
+Note that the Python API does not yet support model save/load but will in the future.
+
{% highlight python %}
from pyspark.mllib.classification import LogisticRegressionWithSGD
from pyspark.mllib.regression import LabeledPoint
@@ -362,7 +344,191 @@ print("Training Error = " + str(trainErr))
-## Linear least squares, Lasso, and ridge regression
+### Logistic regression
+
+[Logistic regression](http://en.wikipedia.org/wiki/Logistic_regression) is widely used to predict a
+binary response. It is a linear method as described above in equation `$\eqref{eq:regPrimal}$`,
+with the loss function in the formulation given by the logistic loss:
+`\[
+L(\wv;\x,y) := \log(1+\exp( -y \wv^T \x)).
+\]`
+
+For binary classification problems, the algorithm outputs a binary logistic regression model.
+Given a new data point, denoted by $\x$, the model makes predictions by
+applying the logistic function
+`\[
+\mathrm{f}(z) = \frac{1}{1 + e^{-z}}
+\]`
+where $z = \wv^T \x$.
+By default, if $\mathrm{f}(\wv^T x) > 0.5$, the outcome is positive, or
+negative otherwise, though unlike linear SVMs, the raw output of the logistic regression
+model, $\mathrm{f}(z)$, has a probabilistic interpretation (i.e., the probability
+that $\x$ is positive).
+
+Binary logistic regression can be generalized into
+[multinomial logistic regression](http://en.wikipedia.org/wiki/Multinomial_logistic_regression) to
+train and predict multiclass classification problems.
+For example, for $K$ possible outcomes, one of the outcomes can be chosen as a "pivot", and the
+other $K - 1$ outcomes can be separately regressed against the pivot outcome.
+In MLlib, the first class $0$ is chosen as the "pivot" class.
+See Section 4.4 of
+[The Elements of Statistical Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for
+references.
+Here is an
+[detailed mathematical derivation](http://www.slideshare.net/dbtsai/2014-0620-mlor-36132297).
+
+For multiclass classification problems, the algorithm will outputs a multinomial logistic regression
+model, which contains $K - 1$ binary logistic regression models regressed against the first class.
+Given a new data points, $K - 1$ models will be run, and the class with largest probability will be
+chosen as the predicted class.
+
+We implemented two algorithms to solve logistic regression: mini-batch gradient descent and L-BFGS.
+We recommend L-BFGS over mini-batch gradient descent for faster convergence.
+
+**Examples**
+
+
+
+
+The following code illustrates how to load a sample multiclass dataset, split it into train and
+test, and use
+[LogisticRegressionWithLBFGS](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS)
+to fit a logistic regression model.
+Then the model is evaluated against the test dataset and saved to disk.
+
+{% highlight scala %}
+import org.apache.spark.SparkContext
+import org.apache.spark.mllib.classification.{LogisticRegressionWithLBFGS, LogisticRegressionModel}
+import org.apache.spark.mllib.evaluation.MulticlassMetrics
+import org.apache.spark.mllib.regression.LabeledPoint
+import org.apache.spark.mllib.linalg.Vectors
+import org.apache.spark.mllib.util.MLUtils
+
+// Load training data in LIBSVM format.
+val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt")
+
+// Split data into training (60%) and test (40%).
+val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L)
+val training = splits(0).cache()
+val test = splits(1)
+
+// Run training algorithm to build the model
+val model = new LogisticRegressionWithLBFGS()
+ .setNumClasses(10)
+ .run(training)
+
+// Compute raw scores on the test set.
+val predictionAndLabels = test.map { case LabeledPoint(label, features) =>
+ val prediction = model.predict(features)
+ (prediction, label)
+}
+
+// Get evaluation metrics.
+val metrics = new MulticlassMetrics(predictionAndLabels)
+val precision = metrics.precision
+println("Precision = " + precision)
+
+// Save and load model
+model.save(sc, "myModelPath")
+val sameModel = LogisticRegressionModel.load(sc, "myModelPath")
+{% endhighlight %}
+
+
+
+
+The following code illustrates how to load a sample multiclass dataset, split it into train and
+test, and use
+[LogisticRegressionWithLBFGS](api/java/org/apache/spark/mllib/classification/LogisticRegressionWithLBFGS.html)
+to fit a logistic regression model.
+Then the model is evaluated against the test dataset and saved to disk.
+
+{% highlight java %}
+import scala.Tuple2;
+
+import org.apache.spark.api.java.*;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.mllib.classification.LogisticRegressionModel;
+import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS;
+import org.apache.spark.mllib.evaluation.MulticlassMetrics;
+import org.apache.spark.mllib.regression.LabeledPoint;
+import org.apache.spark.mllib.util.MLUtils;
+import org.apache.spark.SparkConf;
+import org.apache.spark.SparkContext;
+
+public class MultinomialLogisticRegressionExample {
+ public static void main(String[] args) {
+ SparkConf conf = new SparkConf().setAppName("SVM Classifier Example");
+ SparkContext sc = new SparkContext(conf);
+ String path = "data/mllib/sample_libsvm_data.txt";
+ JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD();
+
+ // Split initial RDD into two... [60% training data, 40% testing data].
+ JavaRDD[] splits = data.randomSplit(new double[] {0.6, 0.4}, 11L);
+ JavaRDD training = splits[0].cache();
+ JavaRDD test = splits[1];
+
+ // Run training algorithm to build the model.
+ final LogisticRegressionModel model = new LogisticRegressionWithLBFGS()
+ .setNumClasses(10)
+ .run(training.rdd());
+
+ // Compute raw scores on the test set.
+ JavaRDD> predictionAndLabels = test.map(
+ new Function>() {
+ public Tuple2
+
+
+The following example shows how to load a sample dataset, build Logistic Regression model,
+and make predictions with the resulting model to compute the training error.
+
+Note that the Python API does not yet support multiclass classification and model save/load but
+will in the future.
+
+{% highlight python %}
+from pyspark.mllib.classification import LogisticRegressionWithLBFGS
+from pyspark.mllib.regression import LabeledPoint
+from numpy import array
+
+# Load and parse the data
+def parsePoint(line):
+ values = [float(x) for x in line.split(' ')]
+ return LabeledPoint(values[0], values[1:])
+
+data = sc.textFile("data/mllib/sample_svm_data.txt")
+parsedData = data.map(parsePoint)
+
+# Build the model
+model = LogisticRegressionWithLBFGS.train(parsedData)
+
+# Evaluating the model on training data
+labelsAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features)))
+trainErr = labelsAndPreds.filter(lambda (v, p): v != p).count() / float(parsedData.count())
+print("Training Error = " + str(trainErr))
+{% endhighlight %}
+
+
+
+# Regression
+
+### Linear least squares, Lasso, and ridge regression
Linear least squares is the most common formulation for regression problems.
@@ -380,7 +546,7 @@ regularization; and [*Lasso*](http://en.wikipedia.org/wiki/Lasso_(statistics)) u
regularization. For all of these models, the average loss or training error, $\frac{1}{n} \sum_{i=1}^n (\wv^T x_i - y_i)^2$, is
known as the [mean squared error](http://en.wikipedia.org/wiki/Mean_squared_error).
-### Examples
+**Examples**
@@ -391,8 +557,9 @@ values. We compute the mean squared error at the end to evaluate
[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit).
{% highlight scala %}
-import org.apache.spark.mllib.regression.LinearRegressionWithSGD
import org.apache.spark.mllib.regression.LabeledPoint
+import org.apache.spark.mllib.regression.LinearRegressionModel
+import org.apache.spark.mllib.regression.LinearRegressionWithSGD
import org.apache.spark.mllib.linalg.Vectors
// Load and parse the data
@@ -413,6 +580,10 @@ val valuesAndPreds = parsedData.map { point =>
}
val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.mean()
println("training Mean Squared Error = " + MSE)
+
+// Save and load model
+model.save(sc, "myModelPath")
+val sameModel = LinearRegressionModel.load(sc, "myModelPath")
{% endhighlight %}
[`RidgeRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD)
@@ -483,6 +654,10 @@ public class LinearRegression {
}
).rdd()).mean();
System.out.println("training Mean Squared Error = " + MSE);
+
+ // Save and load model
+ model.save(sc.sc(), "myModelPath");
+ LinearRegressionModel sameModel = LinearRegressionModel.load(sc.sc(), "myModelPath");
}
}
{% endhighlight %}
@@ -494,6 +669,8 @@ The example then uses LinearRegressionWithSGD to build a simple linear model to
values. We compute the mean squared error at the end to evaluate
[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit).
+Note that the Python API does not yet support model save/load but will in the future.
+
{% highlight python %}
from pyspark.mllib.regression import LabeledPoint, LinearRegressionWithSGD
from numpy import array
@@ -523,7 +700,7 @@ section of the Spark
quick-start guide. Be sure to also include *spark-mllib* to your build file as
a dependency.
-## Streaming linear regression
+###Streaming linear regression
When data arrive in a streaming fashion, it is useful to fit regression models online,
updating the parameters of the model as new data arrives. MLlib currently supports
@@ -531,7 +708,7 @@ streaming linear regression using ordinary least squares. The fitting is similar
to that performed offline, except fitting occurs on each batch of data, so that
the model continually updates to reflect the data from the stream.
-### Examples
+**Examples**
The following example demonstrates how to load training and testing data from two different
input streams of text files, parse the streams as labeled points, fit a linear regression model
@@ -598,7 +775,7 @@ will get better!
-## Implementation (developer)
+# Implementation (developer)
Behind the scene, MLlib implements a simple distributed version of stochastic gradient descent
(SGD), building on the underlying gradient descent primitive (as described in the MLlib - Old Migration Guides
+description: MLlib migration guides from before Spark SPARK_VERSION_SHORT
+---
+
+The migration guide for the current Spark version is kept on the [MLlib Programming Guide main page](mllib-guide.html#migration-guide).
+
+## From 1.1 to 1.2
+
+The only API changes in MLlib v1.2 are in
+[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree),
+which continues to be an experimental API in MLlib 1.2:
+
+1. *(Breaking change)* The Scala API for classification takes a named argument specifying the number
+of classes. In MLlib v1.1, this argument was called `numClasses` in Python and
+`numClassesForClassification` in Scala. In MLlib v1.2, the names are both set to `numClasses`.
+This `numClasses` parameter is specified either via
+[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy)
+or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree)
+static `trainClassifier` and `trainRegressor` methods.
+
+2. *(Breaking change)* The API for
+[`Node`](api/scala/index.html#org.apache.spark.mllib.tree.model.Node) has changed.
+This should generally not affect user code, unless the user manually constructs decision trees
+(instead of using the `trainClassifier` or `trainRegressor` methods).
+The tree `Node` now includes more information, including the probability of the predicted label
+(for classification).
+
+3. Printing methods' output has changed. The `toString` (Scala/Java) and `__repr__` (Python) methods used to print the full model; they now print a summary. For the full model, use `toDebugString`.
+
+Examples in the Spark distribution and examples in the
+[Decision Trees Guide](mllib-decision-tree.html#examples) have been updated accordingly.
+
+## From 1.0 to 1.1
+
+The only API changes in MLlib v1.1 are in
+[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree),
+which continues to be an experimental API in MLlib 1.1:
+
+1. *(Breaking change)* The meaning of tree depth has been changed by 1 in order to match
+the implementations of trees in
+[scikit-learn](http://scikit-learn.org/stable/modules/classes.html#module-sklearn.tree)
+and in [rpart](http://cran.r-project.org/web/packages/rpart/index.html).
+In MLlib v1.0, a depth-1 tree had 1 leaf node, and a depth-2 tree had 1 root node and 2 leaf nodes.
+In MLlib v1.1, a depth-0 tree has 1 leaf node, and a depth-1 tree has 1 root node and 2 leaf nodes.
+This depth is specified by the `maxDepth` parameter in
+[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy)
+or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree)
+static `trainClassifier` and `trainRegressor` methods.
+
+2. *(Non-breaking change)* We recommend using the newly added `trainClassifier` and `trainRegressor`
+methods to build a [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree),
+rather than using the old parameter class `Strategy`. These new training methods explicitly
+separate classification and regression, and they replace specialized parameter types with
+simple `String` types.
+
+Examples of the new, recommended `trainClassifier` and `trainRegressor` are given in the
+[Decision Trees Guide](mllib-decision-tree.html#examples).
+
+## From 0.9 to 1.0
+
+In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few
+breaking changes. If your data is sparse, please store it in a sparse format instead of dense to
+take advantage of sparsity in both storage and computation. Details are described below.
+
diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md
index d5b044d94fdd7..55b8f2ce6c364 100644
--- a/docs/mllib-naive-bayes.md
+++ b/docs/mllib-naive-bayes.md
@@ -37,7 +37,7 @@ smoothing parameter `lambda` as input, and output a
can be used for evaluation and prediction.
{% highlight scala %}
-import org.apache.spark.mllib.classification.NaiveBayes
+import org.apache.spark.mllib.classification.{NaiveBayes, NaiveBayesModel}
import org.apache.spark.mllib.linalg.Vectors
import org.apache.spark.mllib.regression.LabeledPoint
@@ -55,6 +55,10 @@ val model = NaiveBayes.train(training, lambda = 1.0)
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()
+
+// Save and load model
+model.save(sc, "myModelPath")
+val sameModel = NaiveBayesModel.load(sc, "myModelPath")
{% endhighlight %}
@@ -93,6 +97,10 @@ double accuracy = predictionAndLabel.filter(new Function,
return pl._1().equals(pl._2());
}
}).count() / (double) test.count();
+
+// Save and load model
+model.save(sc.sc(), "myModelPath");
+NaiveBayesModel sameModel = NaiveBayesModel.load(sc.sc(), "myModelPath");
{% endhighlight %}
@@ -105,22 +113,30 @@ smoothing parameter `lambda` as input, and output a
[NaiveBayesModel](api/python/pyspark.mllib.classification.NaiveBayesModel-class.html), which can be
used for evaluation and prediction.
-
+Note that the Python API does not yet support model save/load but will in the future.
+
{% highlight python %}
-from pyspark.mllib.regression import LabeledPoint
from pyspark.mllib.classification import NaiveBayes
+from pyspark.mllib.linalg import Vectors
+from pyspark.mllib.regression import LabeledPoint
+
+def parseLine(line):
+ parts = line.split(',')
+ label = float(parts[0])
+ features = Vectors.dense([float(x) for x in parts[1].split(' ')])
+ return LabeledPoint(label, features)
+
+data = sc.textFile('data/mllib/sample_naive_bayes_data.txt').map(parseLine)
-# an RDD of LabeledPoint
-data = sc.parallelize([
- LabeledPoint(0.0, [0.0, 0.0])
- ... # more labeled points
-])
+# Split data aproximately into training (60%) and test (40%)
+training, test = data.randomSplit([0.6, 0.4], seed = 0)
# Train a naive Bayes model.
-model = NaiveBayes.train(data, 1.0)
+model = NaiveBayes.train(training, 1.0)
-# Make prediction.
-prediction = model.predict([0.0, 0.0])
+# Make prediction and test accuracy.
+predictionAndLabel = test.map(lambda p : (model.predict(p.features), p.label))
+accuracy = 1.0 * predictionAndLabel.filter(lambda (x, v): x == v).count() / test.count()
{% endhighlight %}
diff --git a/docs/monitoring.md b/docs/monitoring.md
index f32cdef240d31..6816671ffbf46 100644
--- a/docs/monitoring.md
+++ b/docs/monitoring.md
@@ -1,6 +1,7 @@
---
layout: global
title: Monitoring and Instrumentation
+description: Monitoring, metrics, and instrumentation guide for Spark SPARK_VERSION_SHORT
---
There are several ways to monitor Spark applications: web UIs, metrics, and external instrumentation.
@@ -85,7 +86,7 @@ follows:
- spark.history.fs.updateInterval
+ spark.history.fs.update.interval.seconds
10
The period, in seconds, at which information displayed by this history server is updated.
@@ -144,11 +145,36 @@ follows:
If disabled, no access control checks are made.
+
+ spark.history.fs.cleaner.enabled
+ false
+
+ Specifies whether the History Server should periodically clean up event logs from storage.
+
+
+
+ spark.history.fs.cleaner.interval.seconds
+ 86400
+
+ How often the job history cleaner checks for files to delete, in seconds. Defaults to 86400 (one day).
+ Files are only deleted if they are older than spark.history.fs.cleaner.maxAge.seconds.
+
+
+
+ spark.history.fs.cleaner.maxAge.seconds
+ 3600 * 24 * 7
+
+ Job history files older than this many seconds will be deleted when the history cleaner runs.
+ Defaults to 3600 * 24 * 7 (1 week).
+
+
Note that in all of these UIs, the tables are sortable by clicking their headers,
making it easy to identify slow tasks, data skew, etc.
+Note that the history server only displays completed Spark jobs. One way to signal the completion of a Spark job is to stop the Spark Context explicitly (`sc.stop()`), or in Python using the `with SparkContext() as sc:` to handle the Spark Context setup and tear down, and still show the job history on the UI.
+
# Metrics
Spark has a configurable metrics system based on the
@@ -175,6 +201,7 @@ Each instance can report to zero or more _sinks_. Sinks are contained in the
* `JmxSink`: Registers metrics for viewing in a JMX console.
* `MetricsServlet`: Adds a servlet within the existing Spark UI to serve metrics data as JSON data.
* `GraphiteSink`: Sends metrics to a Graphite node.
+* `Slf4jSink`: Sends metrics to slf4j as log entries.
Spark also supports a Ganglia sink which is not included in the default build due to
licensing restrictions:
diff --git a/docs/programming-guide.md b/docs/programming-guide.md
index 6486614e71354..eda3a95426182 100644
--- a/docs/programming-guide.md
+++ b/docs/programming-guide.md
@@ -1,6 +1,7 @@
---
layout: global
title: Spark Programming Guide
+description: Spark SPARK_VERSION_SHORT programming guide in Java, Scala and Python
---
* This will become a table of contents (this text will be scraped).
@@ -172,8 +173,11 @@ in-process.
In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the
variable called `sc`. Making your own SparkContext will not work. You can set which master the
context connects to using the `--master` argument, and you can add JARs to the classpath
-by passing a comma-separated list to the `--jars` argument.
-For example, to run `bin/spark-shell` on exactly four cores, use:
+by passing a comma-separated list to the `--jars` argument. You can also add dependencies
+(e.g. Spark Packages) to your shell session by supplying a comma-separated list of maven coordinates
+to the `--packages` argument. Any additional repositories where dependencies might exist (e.g. SonaType)
+can be passed to the `--repositories` argument. For example, to run `bin/spark-shell` on exactly
+four cores, use:
{% highlight bash %}
$ ./bin/spark-shell --master local[4]
@@ -185,6 +189,12 @@ Or, to also add `code.jar` to its classpath, use:
$ ./bin/spark-shell --master local[4] --jars code.jar
{% endhighlight %}
+To include a dependency using maven coordinates:
+
+{% highlight bash %}
+$ ./bin/spark-shell --master local[4] --packages "org.example:example:0.1"
+{% endhighlight %}
+
For a complete list of options, run `spark-shell --help`. Behind the scenes,
`spark-shell` invokes the more general [`spark-submit` script](submitting-applications.html).
@@ -195,7 +205,11 @@ For a complete list of options, run `spark-shell --help`. Behind the scenes,
In the PySpark shell, a special interpreter-aware SparkContext is already created for you, in the
variable called `sc`. Making your own SparkContext will not work. You can set which master the
context connects to using the `--master` argument, and you can add Python .zip, .egg or .py files
-to the runtime path by passing a comma-separated list to `--py-files`.
+to the runtime path by passing a comma-separated list to `--py-files`. You can also add dependencies
+(e.g. Spark Packages) to your shell session by supplying a comma-separated list of maven coordinates
+to the `--packages` argument. Any additional repositories where dependencies might exist (e.g. SonaType)
+can be passed to the `--repositories` argument. Any python dependencies a Spark Package has (listed in
+the requirements.txt of that package) must be manually installed using pip when necessary.
For example, to run `bin/pyspark` on exactly four cores, use:
{% highlight bash %}
@@ -321,7 +335,7 @@ Apart from text files, Spark's Scala API also supports several other data format
* For [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), use SparkContext's `sequenceFile[K, V]` method where `K` and `V` are the types of key and values in the file. These should be subclasses of Hadoop's [Writable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Writable.html) interface, like [IntWritable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/IntWritable.html) and [Text](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Text.html). In addition, Spark allows you to specify native types for a few common Writables; for example, `sequenceFile[Int, String]` will automatically read IntWritables and Texts.
-* For other Hadoop InputFormats, you can use the `SparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source. You can also use `SparkContext.newHadoopRDD` for InputFormats based on the "new" MapReduce API (`org.apache.hadoop.mapreduce`).
+* For other Hadoop InputFormats, you can use the `SparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source. You can also use `SparkContext.newAPIHadoopRDD` for InputFormats based on the "new" MapReduce API (`org.apache.hadoop.mapreduce`).
* `RDD.saveAsObjectFile` and `SparkContext.objectFile` support saving an RDD in a simple format consisting of serialized Java objects. While this is not as efficient as specialized formats like Avro, it offers an easy way to save any RDD.
@@ -353,7 +367,7 @@ Apart from text files, Spark's Java API also supports several other data formats
* For [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), use SparkContext's `sequenceFile[K, V]` method where `K` and `V` are the types of key and values in the file. These should be subclasses of Hadoop's [Writable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Writable.html) interface, like [IntWritable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/IntWritable.html) and [Text](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Text.html).
-* For other Hadoop InputFormats, you can use the `JavaSparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source. You can also use `JavaSparkContext.newHadoopRDD` for InputFormats based on the "new" MapReduce API (`org.apache.hadoop.mapreduce`).
+* For other Hadoop InputFormats, you can use the `JavaSparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source. You can also use `JavaSparkContext.newAPIHadoopRDD` for InputFormats based on the "new" MapReduce API (`org.apache.hadoop.mapreduce`).
* `JavaRDD.saveAsObjectFile` and `JavaSparkContext.objectFile` support saving an RDD in a simple format consisting of serialized Java objects. While this is not as efficient as specialized formats like Avro, it offers an easy way to save any RDD.
@@ -711,7 +725,7 @@ class MyClass(object):
def __init__(self):
self.field = "Hello"
def doStuff(self, rdd):
- return rdd.map(lambda s: self.field + x)
+ return rdd.map(lambda s: self.field + s)
{% endhighlight %}
To avoid this issue, the simplest way is to copy `field` into a local variable instead
@@ -720,13 +734,76 @@ of accessing it externally:
{% highlight python %}
def doStuff(self, rdd):
field = self.field
- return rdd.map(lambda s: field + x)
+ return rdd.map(lambda s: field + s)
+{% endhighlight %}
+
+
+
+
+
+### Understanding closures
+One of the harder things about Spark is understanding the scope and life cycle of variables and methods when executing code across a cluster. RDD operations that modify variables outside of their scope can be a frequent source of confusion. In the example below we'll look at code that uses `foreach()` to increment a counter, but similar issues can occur for other operations as well.
+
+#### Example
+
+Consider the naive RDD element sum below, which behaves completely differently depending on whether execution is happening within the same JVM. A common example of this is when running Spark in `local` mode (`--master = local[n]`) versus deploying a Spark application to a cluster (e.g. via spark-submit to YARN):
+
+
+
+
+{% highlight scala %}
+var counter = 0
+var rdd = sc.parallelize(data)
+
+// Wrong: Don't do this!!
+rdd.foreach(x => counter += x)
+
+println("Counter value: " + counter)
+{% endhighlight %}
+
+
+
+{% highlight java %}
+int counter = 0;
+JavaRDD rdd = sc.parallelize(data);
+
+// Wrong: Don't do this!!
+rdd.foreach(x -> counter += x);
+
+println("Counter value: " + counter);
{% endhighlight %}
+
+
+
+{% highlight python %}
+counter = 0
+rdd = sc.parallelize(data)
+
+# Wrong: Don't do this!!
+rdd.foreach(lambda x: counter += x)
+
+print("Counter value: " + counter)
+{% endhighlight %}
+#### Local vs. cluster modes
+
+The primary challenge is that the behavior of the above code is undefined. In local mode with a single JVM, the above code will sum the values within the RDD and store it in **counter**. This is because both the RDD and the variable **counter** are in the same memory space on the driver node.
+
+However, in `cluster` mode, what happens is more complicated, and the above may not work as intended. To execute jobs, Spark breaks up the processing of RDD operations into tasks - each of which is operated on by an executor. Prior to execution, Spark computes the **closure**. The closure is those variables and methods which must be visible for the executor to perform its computations on the RDD (in this case `foreach()`). This closure is serialized and sent to each executor. In `local` mode, there is only the one executors so everything shares the same closure. In other modes however, this is not the case and the executors running on seperate worker nodes each have their own copy of the closure.
+
+What is happening here is that the variables within the closure sent to each executor are now copies and thus, when **counter** is referenced within the `foreach` function, it's no longer the **counter** on the driver node. There is still a **counter** in the memory of the driver node but this is no longer visible to the executors! The executors only sees the copy from the serialized closure. Thus, the final value of **counter** will still be zero since all operations on **counter** were referencing the value within the serialized closure.
+
+To ensure well-defined behavior in these sorts of scenarios one should use an [`Accumulator`](#AccumLink). Accumulators in Spark are used specifically to provide a mechanism for safely updating a variable when execution is split up across worker nodes in a cluster. The Accumulators section of this guide discusses these in more detail.
+
+In general, closures - constructs like loops or locally defined methods, should not be used to mutate some global state. Spark does not define or guarantee the behavior of mutations to objects referenced from outside of closures. Some code that does this may work in local mode, but that's just by accident and such code will not behave as expected in distributed mode. Use an Accumulator instead if some global aggregation is needed.
+
+#### Printing elements of an RDD
+Another common idiom is attempting to print out the elements of an RDD using `rdd.foreach(println)` or `rdd.map(println)`. On a single machine, this will generate the expected output and print all the RDD's elements. However, in `cluster` mode, the output to `stdout` being called by the executors is now writing to the executor's `stdout` instead, not the one on the driver, so `stdout` on the driver won't show these! To print all elements on the driver, one can use the `collect()` method to first bring the RDD to the driver node thus: `rdd.collect().foreach(println)`. This can cause the driver to run out of memory, though, because `collect()` fetches the entire RDD to a single machine; if you only need to print a few elements of the RDD, a safer approach is to use the `take()`: `rdd.take(100).foreach(println)`.
+
### Working with Key-Value Pairs
@@ -974,7 +1051,7 @@ for details.
take(n)
- 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.
+ Return an array with the first n elements of the dataset.
takeSample(withReplacement, num, [seed])
@@ -1004,7 +1081,8 @@ for details.
foreach(func)
- Run a function func on each element of the dataset. This is usually done for side effects such as updating an accumulator variable (see below) or interacting with external storage systems.
+ Run a function func on each element of the dataset. This is usually done for side effects such as updating an Accumulator or interacting with external storage systems.
+
Note: modifying variables other than Accumulators outside of the foreach()
may result in undefined behavior. See Understanding closures for more details.
@@ -1129,6 +1207,12 @@ than shipping a copy of it with tasks. They can be used, for example, to give ev
large input dataset in an efficient manner. Spark also attempts to distribute broadcast variables
using efficient broadcast algorithms to reduce communication cost.
+Spark actions are executed through a set of stages, separated by distributed "shuffle" operations.
+Spark automatically broadcasts the common data needed by tasks within each stage. The data
+broadcasted this way is cached in serialized form and deserialized before running each task. This
+means that explicitly creating broadcast variables is only useful when tasks across multiple stages
+need the same data or when caching the data in deserialized form is important.
+
Broadcast variables are created from a variable `v` by calling `SparkContext.broadcast(v)`. The
broadcast variable is a wrapper around `v`, and its value can be accessed by calling the `value`
method. The code below shows this:
@@ -1177,7 +1261,7 @@ run on the cluster so that `v` is not shipped to the nodes more than once. In ad
`v` should not be modified after it is broadcast in order to ensure that all nodes get the same
value of the broadcast variable (e.g. if the variable is shipped to a new node later).
-## Accumulators
+## Accumulators
Accumulators are variables that are only "added" to through an associative operation and can
therefore be efficiently supported in parallel. They can be used to implement counters (as in
@@ -1322,25 +1406,28 @@ Accumulators do not change the lazy evaluation model of Spark. If they are being
{% highlight scala %}
-val acc = sc.accumulator(0)
-data.map(x => acc += x; f(x))
-// Here, acc is still 0 because no actions have cause the `map` to be computed.
+val accum = sc.accumulator(0)
+data.map { x => accum += x; f(x) }
+// Here, accum is still 0 because no actions have caused the `map` to be computed.
{% endhighlight %}
{% highlight java %}
Accumulator accum = sc.accumulator(0);
-data.map(x -> accum.add(x); f(x););
-// Here, accum is still 0 because no actions have cause the `map` to be computed.
+data.map(x -> { accum.add(x); return f(x); });
+// Here, accum is still 0 because no actions have caused the `map` to be computed.
{% endhighlight %}
{% highlight python %}
accum = sc.accumulator(0)
-data.map(lambda x => acc.add(x); f(x))
-# Here, acc is still 0 because no actions have cause the `map` to be computed.
+def g(x):
+ accum.add(x)
+ return f(x)
+data.map(g)
+# Here, accum is still 0 because no actions have caused the `map` to be computed.
{% endhighlight %}
@@ -1352,6 +1439,11 @@ The [application submission guide](submitting-applications.html) describes how t
In short, once you package your application into a JAR (for Java/Scala) or a set of `.py` or `.zip` files (for Python),
the `bin/spark-submit` script lets you submit it to any supported cluster manager.
+# Launching Spark jobs from Java / Scala
+
+The [org.apache.spark.launcher](api/java/index.html?org/apache/spark/launcher/package-summary.html)
+package provides classes for launching Spark jobs as child processes using a simple Java API.
+
# Unit Testing
Spark is friendly to unit testing with any popular unit test framework.
diff --git a/docs/quick-start.md b/docs/quick-start.md
index bf643bb70e153..81143da865cf0 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -1,6 +1,7 @@
---
layout: global
title: Quick Start
+description: Quick start tutorial for Spark SPARK_VERSION_SHORT
---
* This will become a table of contents (this text will be scraped).
diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md
index 78358499fd01f..59a3e9d25baf1 100644
--- a/docs/running-on-mesos.md
+++ b/docs/running-on-mesos.md
@@ -110,7 +110,7 @@ cluster, or `mesos://zk://host:2181` for a multi-master Mesos cluster using ZooK
The driver also needs some configuration in `spark-env.sh` to interact properly with Mesos:
1. In `spark-env.sh` set some environment variables:
- * `export MESOS_NATIVE_LIBRARY=`. This path is typically
+ * `export MESOS_NATIVE_JAVA_LIBRARY=`. This path is typically
`/lib/libmesos.so` where the prefix is `/usr/local` by default. See Mesos installation
instructions above. On Mac OS X, the library is called `libmesos.dylib` instead of
`libmesos.so`.
@@ -167,9 +167,6 @@ acquire. By default, it will acquire *all* cores in the cluster (that get offere
only makes sense if you run just one application at a time. You can cap the maximum number of cores
using `conf.set("spark.cores.max", "10")` (for example).
-# Known issues
-- When using the "fine-grained" mode, make sure that your executors always leave 32 MB free on the slaves. Otherwise it can happen that your Spark job does not proceed anymore. Currently, Apache Mesos only offers resources if there are at least 32 MB memory allocatable. But as Spark allocates memory only for the executor and cpu only for tasks, it can happen on high slave memory usage that no new tasks will be started anymore. More details can be found in [MESOS-1688](https://issues.apache.org/jira/browse/MESOS-1688). Alternatively use the "coarse-gained" mode, which is not affected by this issue.
-
# Running Alongside Hadoop
You can run Spark and Mesos alongside your existing Hadoop cluster by just launching them as a
@@ -197,7 +194,11 @@ See the [configuration page](configuration.html) for information on Spark config
spark.mesos.coarse
false
- Set the run mode for Spark on Mesos. For more information about the run mode, refer to #Mesos Run Mode section above.
+ If set to "true", runs over Mesos clusters in
+ "coarse-grained" sharing mode,
+ where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per
+ Spark task. This gives lower-latency scheduling for short queries, but leaves resources in use
+ for the whole duration of the Spark job.
@@ -211,19 +212,23 @@ See the [configuration page](configuration.html) for information on Spark config
spark.mesos.executor.home
- SPARK_HOME
+ driver side SPARK_HOME
- The location where the mesos executor will look for Spark binaries to execute, and uses the SPARK_HOME setting on default.
- This variable is only used when no spark.executor.uri is provided, and assumes Spark is installed on the specified location
- on each slave.
+ Set the directory in which Spark is installed on the executors in Mesos. By default, the
+ executors will simply use the driver's Spark home directory, which may not be visible to
+ them. Note that this is only relevant if a Spark binary package is not specified through
+ spark.executor.uri
.
spark.mesos.executor.memoryOverhead
- 384
+ executor memory * 0.10, with minimum of 384
- The amount of memory that Mesos executor will request for the task to account for the overhead of running the executor itself.
- The final total amount of memory allocated is the maximum value between executor memory plus memoryOverhead, and overhead fraction (1.07) plus the executor memory.
+ This value is an additive for spark.executor.memory
, specified in MiB,
+ which is used to calculate the total Mesos task memory. A value of 384
+ implies a 384MiB overhead. Additionally, there is a hard-coded 7% minimum
+ overhead. The final overhead will be the larger of either
+ `spark.mesos.executor.memoryOverhead` or 7% of `spark.executor.memory`.
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 68ab127bcf087..68b1aeb8ebd01 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -104,9 +104,16 @@ Most of the configs are the same for Spark on YARN as for other deployment modes
Comma-separated list of files to be placed in the working directory of each executor.
+
+ spark.executor.instances
+ 2
+
+ The number of executors. Note that this property is incompatible with spark.dynamicAllocation.enabled
.
+
+
spark.yarn.executor.memoryOverhead
- executorMemory * 0.07, with minimum of 384
+ executorMemory * 0.10, with minimum of 384
The amount of off heap memory (in megabytes) to be allocated per executor. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the executor size (typically 6-10%).
diff --git a/docs/security.md b/docs/security.md
index 1e206a139fb72..c034ba12ff1fc 100644
--- a/docs/security.md
+++ b/docs/security.md
@@ -1,6 +1,7 @@
---
layout: global
-title: Spark Security
+displayTitle: Spark Security
+title: Security
---
Spark currently supports authentication via a shared secret. Authentication can be configured to be on via the `spark.authenticate` configuration parameter. This parameter controls whether the Spark communication protocols do authentication using the shared secret. This authentication is a basic handshake to make sure both sides have the same shared secret and are allowed to communicate. If the shared secret is not identical they will not be allowed to communicate. The shared secret is created as follows:
@@ -20,6 +21,30 @@ Spark allows for a set of administrators to be specified in the acls who always
If your applications are using event logging, the directory where the event logs go (`spark.eventLog.dir`) should be manually created and have the proper permissions set on it. If you want those log files secured, the permissions should be set to `drwxrwxrwxt` for that directory. The owner of the directory should be the super user who is running the history server and the group permissions should be restricted to super user group. This will allow all users to write to the directory but will prevent unprivileged users from removing or renaming a file unless they own the file or directory. The event log files will be created by Spark with permissions such that only the user and group have read and write access.
+## Encryption
+
+Spark supports SSL for Akka and HTTP (for broadcast and file server) protocols. However SSL is not supported yet for WebUI and block transfer service.
+
+Connection encryption (SSL) configuration is organized hierarchically. The user can configure the default SSL settings which will be used for all the supported communication protocols unless they are overwritten by protocol-specific settings. This way the user can easily provide the common settings for all the protocols without disabling the ability to configure each one individually. The common SSL settings are at `spark.ssl` namespace in Spark configuration, while Akka SSL configuration is at `spark.ssl.akka` and HTTP for broadcast and file server SSL configuration is at `spark.ssl.fs`. The full breakdown can be found on the [configuration page](configuration.html).
+
+SSL must be configured on each node and configured for each component involved in communication using the particular protocol.
+
+### YARN mode
+The key-store can be prepared on the client side and then distributed and used by the executors as the part of the application. It is possible because the user is able to deploy files before the application is started in YARN by using `spark.yarn.dist.files` or `spark.yarn.dist.archives` configuration settings. The responsibility for encryption of transferring these files is on YARN side and has nothing to do with Spark.
+
+### Standalone mode
+The user needs to provide key-stores and configuration options for master and workers. They have to be set by attaching appropriate Java system properties in `SPARK_MASTER_OPTS` and in `SPARK_WORKER_OPTS` environment variables, or just in `SPARK_DAEMON_JAVA_OPTS`. In this mode, the user may allow the executors to use the SSL settings inherited from the worker which spawned that executor. It can be accomplished by setting `spark.ssl.useNodeLocalConf` to `true`. If that parameter is set, the settings provided by user on the client side, are not used by the executors.
+
+### Preparing the key-stores
+Key-stores can be generated by `keytool` program. The reference documentation for this tool is
+[here](https://docs.oracle.com/javase/7/docs/technotes/tools/solaris/keytool.html). The most basic
+steps to configure the key-stores and the trust-store for the standalone deployment mode is as
+follows:
+* Generate a keys pair for each node
+* Export the public key of the key pair to a file on each node
+* Import all exported public keys into a single trust-store
+* Distribute the trust-store over the nodes
+
## Configuring Ports for Network Security
Spark makes heavy use of the network, and some environments have strict requirements for using tight
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 5c6084fb46255..74d8653a8b845 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -222,8 +222,7 @@ SPARK_WORKER_OPTS supports the following system properties:
false
Enable periodic cleanup of worker / application directories. Note that this only affects standalone
- mode, as YARN works differently. Applications directories are cleaned up regardless of whether
- the application is still running.
+ mode, as YARN works differently. Only the directories of stopped applications are cleaned up.
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index be8c5c2c1522e..11c29e20632ae 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -1,6 +1,7 @@
---
layout: global
-title: Spark SQL Programming Guide
+displayTitle: Spark SQL and DataFrame Guide
+title: Spark SQL and DataFrames
---
* This will become a table of contents (this text will be scraped).
@@ -8,168 +9,348 @@ title: Spark SQL Programming Guide
# Overview
+Spark SQL is a Spark module for structured data processing. It provides a programming abstraction called DataFrames and can also act as distributed SQL query engine.
+
+
+# DataFrames
+
+A DataFrame is a distributed collection of data organized into named columns. It is conceptually equivalent to a table in a relational database or a data frame in R/Python, but with richer optimizations under the hood. DataFrames can be constructed from a wide array of sources such as: structured data files, tables in Hive, external databases, or existing RDDs.
+
+The DataFrame API is available in [Scala](api/scala/index.html#org.apache.spark.sql.DataFrame), [Java](api/java/index.html?org/apache/spark/sql/DataFrame.html), and [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame).
+
+All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell` or the `pyspark` shell.
+
+
+## Starting Point: `SQLContext`
+
-Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using
-Spark. At the core of this component is a new type of RDD,
-[SchemaRDD](api/scala/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed of
-[Row](api/scala/index.html#org.apache.spark.sql.package@Row:org.apache.spark.sql.catalyst.expressions.Row.type) objects, along with
-a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table
-in a traditional relational database. A SchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io)
-file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/).
+The entry point into all functionality in Spark SQL is the
+[`SQLContext`](api/scala/index.html#org.apache.spark.sql.`SQLContext`) class, or one of its
+descendants. To create a basic `SQLContext`, all you need is a SparkContext.
+
+{% highlight scala %}
+val sc: SparkContext // An existing SparkContext.
+val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell`.
+// this is used to implicitly convert an RDD to a DataFrame.
+import sqlContext.implicits._
+{% endhighlight %}
-
-Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using
-Spark. At the core of this component is a new type of RDD,
-[JavaSchemaRDD](api/scala/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed of
-[Row](api/scala/index.html#org.apache.spark.sql.api.java.Row) objects, along with
-a schema that describes the data types of each column in the row. A JavaSchemaRDD is similar to a table
-in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io)
-file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/).
+
+
+The entry point into all functionality in Spark SQL is the
+[`SQLContext`](api/java/index.html#org.apache.spark.sql.SQLContext) class, or one of its
+descendants. To create a basic `SQLContext`, all you need is a SparkContext.
+
+{% highlight java %}
+JavaSparkContext sc = ...; // An existing JavaSparkContext.
+SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
+{% endhighlight %}
+
-Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using
-Spark. At the core of this component is a new type of RDD,
-[SchemaRDD](api/python/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed of
-[Row](api/python/pyspark.sql.Row-class.html) objects, along with
-a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table
-in a traditional relational database. A SchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io)
-file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/).
+The entry point into all relational functionality in Spark is the
+[`SQLContext`](api/python/pyspark.sql.SQLContext-class.html) class, or one
+of its decedents. To create a basic `SQLContext`, all you need is a SparkContext.
+
+{% highlight python %}
+from pyspark.sql import SQLContext
+sqlContext = SQLContext(sc)
+{% endhighlight %}
-All of the examples on this page use sample data included in the Spark distribution and can be run in the `pyspark` shell.
-**Spark SQL is currently an alpha component. While we will minimize API changes, some APIs may change in future releases.**
+In addition to the basic `SQLContext`, you can also create a `HiveContext`, which provides a
+superset of the functionality provided by the basic `SQLContext`. Additional features include
+the ability to write queries using the more complete HiveQL parser, access to Hive UDFs, and the
+ability to read data from Hive tables. To use a `HiveContext`, you do not need to have an
+existing Hive setup, and all of the data sources available to a `SQLContext` are still available.
+`HiveContext` is only packaged separately to avoid including all of Hive's dependencies in the default
+Spark build. If these dependencies are not a problem for your application then using `HiveContext`
+is recommended for the 1.3 release of Spark. Future releases will focus on bringing `SQLContext` up
+to feature parity with a `HiveContext`.
+
+The specific variant of SQL that is used to parse queries can also be selected using the
+`spark.sql.dialect` option. This parameter can be changed using either the `setConf` method on
+a `SQLContext` or by using a `SET key=value` command in SQL. For a `SQLContext`, the only dialect
+available is "sql" which uses a simple SQL parser provided by Spark SQL. In a `HiveContext`, the
+default is "hiveql", though "sql" is also available. Since the HiveQL parser is much more complete,
+this is recommended for most use cases.
+
+
+## Creating DataFrames
-***************************************************************************************************
+With a `SQLContext`, applications can create `DataFrame`s from an existing `RDD`, from a Hive table, or from data sources.
-# Getting Started
+As an example, the following creates a `DataFrame` based on the content of a JSON file:
-
-The entry point into all relational functionality in Spark is the
-[SQLContext](api/scala/index.html#org.apache.spark.sql.SQLContext) class, or one of its
-descendants. To create a basic SQLContext, all you need is a SparkContext.
-
{% highlight scala %}
val sc: SparkContext // An existing SparkContext.
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-// createSchemaRDD is used to implicitly convert an RDD to a SchemaRDD.
-import sqlContext.createSchemaRDD
-{% endhighlight %}
+val df = sqlContext.jsonFile("examples/src/main/resources/people.json")
-In addition to the basic SQLContext, you can also create a HiveContext, which provides a
-superset of the functionality provided by the basic SQLContext. Additional features include
-the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the
-ability to read data from Hive tables. To use a HiveContext, you do not need to have an
-existing Hive setup, and all of the data sources available to a SQLContext are still available.
-HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default
-Spark build. If these dependencies are not a problem for your application then using HiveContext
-is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to
-feature parity with a HiveContext.
+// Displays the content of the DataFrame to stdout
+df.show()
+{% endhighlight %}
-
-The entry point into all relational functionality in Spark is the
-[JavaSQLContext](api/scala/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one
-of its descendants. To create a basic JavaSQLContext, all you need is a JavaSparkContext.
-
{% highlight java %}
JavaSparkContext sc = ...; // An existing JavaSparkContext.
-JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc);
-{% endhighlight %}
+SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
-In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict
-super set of the functionality provided by the basic SQLContext. Additional features include
-the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the
-ability to read data from Hive tables. To use a HiveContext, you do not need to have an
-existing Hive setup, and all of the data sources available to a SQLContext are still available.
-HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default
-Spark build. If these dependencies are not a problem for your application then using HiveContext
-is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to
-feature parity with a HiveContext.
+DataFrame df = sqlContext.jsonFile("examples/src/main/resources/people.json");
+
+// Displays the content of the DataFrame to stdout
+df.show();
+{% endhighlight %}
+{% highlight python %}
+from pyspark.sql import SQLContext
+sqlContext = SQLContext(sc)
-The entry point into all relational functionality in Spark is the
-[SQLContext](api/python/pyspark.sql.SQLContext-class.html) class, or one
-of its decedents. To create a basic SQLContext, all you need is a SparkContext.
+df = sqlContext.jsonFile("examples/src/main/resources/people.json")
+# Displays the content of the DataFrame to stdout
+df.show()
+{% endhighlight %}
+
+
+
+
+
+## DataFrame Operations
+
+DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.DataFrame), [Java](api/java/index.html?org/apache/spark/sql/DataFrame.html), and [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame).
+
+Here we include some basic examples of structured data processing using DataFrames:
+
+
+
+
+{% highlight scala %}
+val sc: SparkContext // An existing SparkContext.
+val sqlContext = new org.apache.spark.sql.SQLContext(sc)
+
+// Create the DataFrame
+val df = sqlContext.jsonFile("examples/src/main/resources/people.json")
+
+// Show the content of the DataFrame
+df.show()
+// age name
+// null Michael
+// 30 Andy
+// 19 Justin
+
+// Print the schema in a tree format
+df.printSchema()
+// root
+// |-- age: long (nullable = true)
+// |-- name: string (nullable = true)
+
+// Select only the "name" column
+df.select("name").show()
+// name
+// Michael
+// Andy
+// Justin
+
+// Select everybody, but increment the age by 1
+df.select("name", df("age") + 1).show()
+// name (age + 1)
+// Michael null
+// Andy 31
+// Justin 20
+
+// Select people older than 21
+df.filter(df("name") > 21).show()
+// age name
+// 30 Andy
+
+// Count people by age
+df.groupBy("age").count().show()
+// age count
+// null 1
+// 19 1
+// 30 1
+{% endhighlight %}
+
+
+
+
+{% highlight java %}
+val sc: JavaSparkContext // An existing SparkContext.
+val sqlContext = new org.apache.spark.sql.SQLContext(sc)
+
+// Create the DataFrame
+DataFrame df = sqlContext.jsonFile("examples/src/main/resources/people.json");
+
+// Show the content of the DataFrame
+df.show();
+// age name
+// null Michael
+// 30 Andy
+// 19 Justin
+
+// Print the schema in a tree format
+df.printSchema();
+// root
+// |-- age: long (nullable = true)
+// |-- name: string (nullable = true)
+
+// Select only the "name" column
+df.select("name").show();
+// name
+// Michael
+// Andy
+// Justin
+
+// Select everybody, but increment the age by 1
+df.select("name", df.col("age").plus(1)).show();
+// name (age + 1)
+// Michael null
+// Andy 31
+// Justin 20
+
+// Select people older than 21
+df.filter(df("name") > 21).show();
+// age name
+// 30 Andy
+
+// Count people by age
+df.groupBy("age").count().show();
+// age count
+// null 1
+// 19 1
+// 30 1
+{% endhighlight %}
+
+
+
+
{% highlight python %}
from pyspark.sql import SQLContext
sqlContext = SQLContext(sc)
-{% endhighlight %}
-In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict
-super set of the functionality provided by the basic SQLContext. Additional features include
-the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the
-ability to read data from Hive tables. To use a HiveContext, you do not need to have an
-existing Hive setup, and all of the data sources available to a SQLContext are still available.
-HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default
-Spark build. If these dependencies are not a problem for your application then using HiveContext
-is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to
-feature parity with a HiveContext.
+# Create the DataFrame
+df = sqlContext.jsonFile("examples/src/main/resources/people.json")
+
+# Show the content of the DataFrame
+df.show()
+## age name
+## null Michael
+## 30 Andy
+## 19 Justin
+
+# Print the schema in a tree format
+df.printSchema()
+## root
+## |-- age: long (nullable = true)
+## |-- name: string (nullable = true)
+
+# Select only the "name" column
+df.select("name").show()
+## name
+## Michael
+## Andy
+## Justin
+
+# Select everybody, but increment the age by 1
+df.select("name", df.age + 1).show()
+## name (age + 1)
+## Michael null
+## Andy 31
+## Justin 20
+
+# Select people older than 21
+df.filter(df.name > 21).show()
+## age name
+## 30 Andy
+
+# Count people by age
+df.groupBy("age").count().show()
+## age count
+## null 1
+## 19 1
+## 30 1
+{% endhighlight %}
+
+
+
+## Running SQL Queries Programmatically
+
+The `sql` function on a `SQLContext` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`.
+
+
+
+{% highlight scala %}
+val sqlContext = ... // An existing SQLContext
+val df = sqlContext.sql("SELECT * FROM table")
+{% endhighlight %}
-The specific variant of SQL that is used to parse queries can also be selected using the
-`spark.sql.dialect` option. This parameter can be changed using either the `setConf` method on
-a SQLContext or by using a `SET key=value` command in SQL. For a SQLContext, the only dialect
-available is "sql" which uses a simple SQL parser provided by Spark SQL. In a HiveContext, the
-default is "hiveql", though "sql" is also available. Since the HiveQL parser is much more complete,
- this is recommended for most use cases.
+
+{% highlight java %}
+val sqlContext = ... // An existing SQLContext
+val df = sqlContext.sql("SELECT * FROM table")
+{% endhighlight %}
+
-# Data Sources
+
+{% highlight python %}
+from pyspark.sql import SQLContext
+sqlContext = SQLContext(sc)
+df = sqlContext.sql("SELECT * FROM table")
+{% endhighlight %}
+
+
-Spark SQL supports operating on a variety of data sources through the `SchemaRDD` interface.
-A SchemaRDD can be operated on as normal RDDs and can also be registered as a temporary table.
-Registering a SchemaRDD as a table allows you to run SQL queries over its data. This section
-describes the various methods for loading data into a SchemaRDD.
-## RDDs
+## Interoperating with RDDs
-Spark SQL supports two different methods for converting existing RDDs into SchemaRDDs. The first
+Spark SQL supports two different methods for converting existing RDDs into DataFrames. The first
method uses reflection to infer the schema of an RDD that contains specific types of objects. This
reflection based approach leads to more concise code and works well when you already know the schema
while writing your Spark application.
-The second method for creating SchemaRDDs is through a programmatic interface that allows you to
+The second method for creating DataFrames is through a programmatic interface that allows you to
construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows
-you to construct SchemaRDDs when the columns and their types are not known until runtime.
+you to construct DataFrames when the columns and their types are not known until runtime.
### Inferring the Schema Using Reflection
-The Scala interaface for Spark SQL supports automatically converting an RDD containing case classes
-to a SchemaRDD. The case class
+The Scala interface for Spark SQL supports automatically converting an RDD containing case classes
+to a DataFrame. The case class
defines the schema of the table. The names of the arguments to the case class are read using
reflection and become the names of the columns. Case classes can also be nested or contain complex
-types such as Sequences or Arrays. This RDD can be implicitly converted to a SchemaRDD and then be
+types such as Sequences or Arrays. This RDD can be implicitly converted to a DataFrame and then be
registered as a table. Tables can be used in subsequent SQL statements.
{% highlight scala %}
// sc is an existing SparkContext.
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-// createSchemaRDD is used to implicitly convert an RDD to a SchemaRDD.
-import sqlContext.createSchemaRDD
+// this is used to implicitly convert an RDD to a DataFrame.
+import sqlContext.implicits._
// Define the schema using a case class.
// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit,
@@ -177,13 +358,13 @@ import sqlContext.createSchemaRDD
case class Person(name: String, age: Int)
// Create an RDD of Person objects and register it as a table.
-val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt))
+val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)).toDF()
people.registerTempTable("people")
// SQL statements can be run by using the sql methods provided by sqlContext.
val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19")
-// The results of SQL queries are SchemaRDDs and support all the normal RDD operations.
+// The results of SQL queries are DataFrames and support all the normal RDD operations.
// The columns of a row in the result can be accessed by ordinal.
teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
{% endhighlight %}
@@ -193,7 +374,7 @@ teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
Spark SQL supports automatically converting an RDD of [JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly)
-into a Schema RDD. The BeanInfo, obtained using reflection, defines the schema of the table.
+into a DataFrame. The BeanInfo, obtained using reflection, defines the schema of the table.
Currently, Spark SQL does not support JavaBeans that contain
nested or contain complex types such as Lists or Arrays. You can create a JavaBean by creating a
class that implements Serializable and has getters and setters for all of its fields.
@@ -224,12 +405,12 @@ public static class Person implements Serializable {
{% endhighlight %}
-A schema can be applied to an existing RDD by calling `applySchema` and providing the Class object
+A schema can be applied to an existing RDD by calling `createDataFrame` and providing the Class object
for the JavaBean.
{% highlight java %}
// sc is an existing JavaSparkContext.
-JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc);
+SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
// Load a text file and convert each line to a JavaBean.
JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").map(
@@ -246,13 +427,13 @@ JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").m
});
// Apply a schema to an RDD of JavaBeans and register it as a table.
-JavaSchemaRDD schemaPeople = sqlContext.applySchema(people, Person.class);
+DataFrame schemaPeople = sqlContext.createDataFrame(people, Person.class);
schemaPeople.registerTempTable("people");
// SQL can be run over RDDs that have been registered as tables.
-JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19")
+DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19")
-// The results of SQL queries are SchemaRDDs and support all the normal RDD operations.
+// The results of SQL queries are DataFrames and support all the normal RDD operations.
// The columns of a row in the result can be accessed by ordinal.
List teenagerNames = teenagers.map(new Function() {
public String call(Row row) {
@@ -266,7 +447,7 @@ List teenagerNames = teenagers.map(new Function() {
-Spark SQL can convert an RDD of Row objects to a SchemaRDD, inferring the datatypes. Rows are constructed by passing a list of
+Spark SQL can convert an RDD of Row objects to a DataFrame, inferring the datatypes. Rows are constructed by passing a list of
key/value pairs as kwargs to the Row class. The keys of this list define the column names of the table,
and the types are inferred by looking at the first row. Since we currently only look at the first
row, it is important that there is no missing data in the first row of the RDD. In future versions we
@@ -283,11 +464,11 @@ lines = sc.textFile("examples/src/main/resources/people.txt")
parts = lines.map(lambda l: l.split(","))
people = parts.map(lambda p: Row(name=p[0], age=int(p[1])))
-# Infer the schema, and register the SchemaRDD as a table.
+# Infer the schema, and register the DataFrame as a table.
schemaPeople = sqlContext.inferSchema(people)
schemaPeople.registerTempTable("people")
-# SQL can be run over SchemaRDDs that have been registered as a table.
+# SQL can be run over DataFrames that have been registered as a table.
teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19")
# The results of SQL queries are RDDs and support all the normal RDD operations.
@@ -309,12 +490,12 @@ for teenName in teenNames.collect():
When case classes cannot be defined ahead of time (for example,
the structure of records is encoded in a string, or a text dataset will be parsed
and fields will be projected differently for different users),
-a `SchemaRDD` can be created programmatically with three steps.
+a `DataFrame` can be created programmatically with three steps.
1. Create an RDD of `Row`s from the original RDD;
2. Create the schema represented by a `StructType` matching the structure of
`Row`s in the RDD created in Step 1.
-3. Apply the schema to the RDD of `Row`s via `applySchema` method provided
+3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided
by `SQLContext`.
For example:
@@ -340,15 +521,15 @@ val schema =
val rowRDD = people.map(_.split(",")).map(p => Row(p(0), p(1).trim))
// Apply the schema to the RDD.
-val peopleSchemaRDD = sqlContext.applySchema(rowRDD, schema)
+val peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema)
-// Register the SchemaRDD as a table.
-peopleSchemaRDD.registerTempTable("people")
+// Register the DataFrames as a table.
+peopleDataFrame.registerTempTable("people")
// SQL statements can be run by using the sql methods provided by sqlContext.
val results = sqlContext.sql("SELECT name FROM people")
-// The results of SQL queries are SchemaRDDs and support all the normal RDD operations.
+// The results of SQL queries are DataFrames and support all the normal RDD operations.
// The columns of a row in the result can be accessed by ordinal.
results.map(t => "Name: " + t(0)).collect().foreach(println)
{% endhighlight %}
@@ -361,26 +542,26 @@ results.map(t => "Name: " + t(0)).collect().foreach(println)
When JavaBean classes cannot be defined ahead of time (for example,
the structure of records is encoded in a string, or a text dataset will be parsed and
fields will be projected differently for different users),
-a `SchemaRDD` can be created programmatically with three steps.
+a `DataFrame` can be created programmatically with three steps.
1. Create an RDD of `Row`s from the original RDD;
2. Create the schema represented by a `StructType` matching the structure of
`Row`s in the RDD created in Step 1.
-3. Apply the schema to the RDD of `Row`s via `applySchema` method provided
-by `JavaSQLContext`.
+3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided
+by `SQLContext`.
For example:
{% highlight java %}
// Import factory methods provided by DataType.
-import org.apache.spark.sql.api.java.DataType
+import org.apache.spark.sql.types.DataType;
// Import StructType and StructField
-import org.apache.spark.sql.api.java.StructType
-import org.apache.spark.sql.api.java.StructField
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.types.StructField;
// Import Row.
-import org.apache.spark.sql.api.java.Row
+import org.apache.spark.sql.Row;
// sc is an existing JavaSparkContext.
-JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc);
+SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
// Load a text file and convert each line to a JavaBean.
JavaRDD people = sc.textFile("examples/src/main/resources/people.txt");
@@ -405,15 +586,15 @@ JavaRDD rowRDD = people.map(
});
// Apply the schema to the RDD.
-JavaSchemaRDD peopleSchemaRDD = sqlContext.applySchema(rowRDD, schema);
+DataFrame peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema);
-// Register the SchemaRDD as a table.
-peopleSchemaRDD.registerTempTable("people");
+// Register the DataFrame as a table.
+peopleDataFrame.registerTempTable("people");
// SQL can be run over RDDs that have been registered as tables.
-JavaSchemaRDD results = sqlContext.sql("SELECT name FROM people");
+DataFrame results = sqlContext.sql("SELECT name FROM people");
-// The results of SQL queries are SchemaRDDs and support all the normal RDD operations.
+// The results of SQL queries are DataFrames and support all the normal RDD operations.
// The columns of a row in the result can be accessed by ordinal.
List names = results.map(new Function() {
public String call(Row row) {
@@ -430,12 +611,12 @@ List names = results.map(new Function() {
When a dictionary of kwargs cannot be defined ahead of time (for example,
the structure of records is encoded in a string, or a text dataset will be parsed and
fields will be projected differently for different users),
-a `SchemaRDD` can be created programmatically with three steps.
+a `DataFrame` can be created programmatically with three steps.
1. Create an RDD of tuples or lists from the original RDD;
2. Create the schema represented by a `StructType` matching the structure of
tuples or lists in the RDD created in the step 1.
-3. Apply the schema to the RDD via `applySchema` method provided by `SQLContext`.
+3. Apply the schema to the RDD via `createDataFrame` method provided by `SQLContext`.
For example:
{% highlight python %}
@@ -457,12 +638,12 @@ fields = [StructField(field_name, StringType(), True) for field_name in schemaSt
schema = StructType(fields)
# Apply the schema to the RDD.
-schemaPeople = sqlContext.applySchema(people, schema)
+schemaPeople = sqlContext.createDataFrame(people, schema)
-# Register the SchemaRDD as a table.
+# Register the DataFrame as a table.
schemaPeople.registerTempTable("people")
-# SQL can be run over SchemaRDDs that have been registered as a table.
+# SQL can be run over DataFrames that have been registered as a table.
results = sqlContext.sql("SELECT name FROM people")
# The results of SQL queries are RDDs and support all the normal RDD operations.
@@ -471,11 +652,157 @@ for name in names.collect():
print name
{% endhighlight %}
+
+
+
+
+
+# Data Sources
+
+Spark SQL supports operating on a variety of data sources through the `DataFrame` interface.
+A DataFrame can be operated on as normal RDDs and can also be registered as a temporary table.
+Registering a DataFrame as a table allows you to run SQL queries over its data. This section
+describes the general methods for loading and saving data using the Spark Data Sources and then
+goes into specific options that are available for the built-in data sources.
+
+## Generic Load/Save Functions
+
+In the simplest form, the default data source (`parquet` unless otherwise configured by
+`spark.sql.sources.default`) will be used for all operations.
+
+
+
+
+{% highlight scala %}
+val df = sqlContext.load("people.parquet")
+df.select("name", "age").save("namesAndAges.parquet")
+{% endhighlight %}
+
+
+
+
+
+{% highlight java %}
+
+DataFrame df = sqlContext.load("people.parquet");
+df.select("name", "age").save("namesAndAges.parquet");
+
+{% endhighlight %}
+
+
+
+
+
+{% highlight python %}
+
+df = sqlContext.load("people.parquet")
+df.select("name", "age").save("namesAndAges.parquet")
+
+{% endhighlight %}
+
+
+
+
+### Manually Specifying Options
+
+You can also manually specify the data source that will be used along with any extra options
+that you would like to pass to the data source. Data sources are specified by their fully qualified
+name (i.e., `org.apache.spark.sql.parquet`), but for built-in sources you can also use the shorted
+name (`json`, `parquet`, `jdbc`). DataFrames of any type can be converted into other types
+using this syntax.
+
+
+
+
+{% highlight scala %}
+val df = sqlContext.load("people.json", "json")
+df.select("name", "age").save("namesAndAges.parquet", "parquet")
+{% endhighlight %}
+
+
+{% highlight java %}
+
+DataFrame df = sqlContext.load("people.json", "json");
+df.select("name", "age").save("namesAndAges.parquet", "parquet");
+
+{% endhighlight %}
+
+
+
+{% highlight python %}
+
+df = sqlContext.load("people.json", "json")
+df.select("name", "age").save("namesAndAges.parquet", "parquet")
+
+{% endhighlight %}
+
+
+
+
+### Save Modes
+
+Save operations can optionally take a `SaveMode`, that specifies how to handle existing data if
+present. It is important to realize that these save modes do not utilize any locking and are not
+atomic. Thus, it is not safe to have multiple writers attempting to write to the same location.
+Additionally, when performing a `Overwrite`, the data will be deleted before writing out the
+new data.
+
+
+Scala/Java Python Meaning
+
+ SaveMode.ErrorIfExists
(default)
+ "error"
(default)
+
+ When saving a DataFrame to a data source, if data already exists,
+ an exception is expected to be thrown.
+
+
+
+ SaveMode.Append
+ "append"
+
+ When saving a DataFrame to a data source, if data/table already exists,
+ contents of the DataFrame are expected to be appended to existing data.
+
+
+
+ SaveMode.Overwrite
+ "overwrite"
+
+ Overwrite mode means that when saving a DataFrame to a data source,
+ if data/table already exists, existing data is expected to be overwritten by the contents of
+ the DataFrame.
+
+
+
+ SaveMode.Ignore
+ "ignore"
+
+ Ignore mode means that when saving a DataFrame to a data source, if data already exists,
+ the save operation is expected to not save the contents of the DataFrame and to not
+ change the existing data. This is similar to a `CREATE TABLE IF NOT EXISTS` in SQL.
+
+
+
+
+### Saving to Persistent Tables
+
+When working with a `HiveContext`, `DataFrames` can also be saved as persistent tables using the
+`saveAsTable` command. Unlike the `registerTempTable` command, `saveAsTable` will materialize the
+contents of the dataframe and create a pointer to the data in the HiveMetastore. Persistent tables
+will still exist even after your Spark program has restarted, as long as you maintain your connection
+to the same metastore. A DataFrame for a persistent table can be created by calling the `table`
+method on a `SQLContext` with the name of the table.
+
+By default `saveAsTable` will create a "managed table", meaning that the location of the data will
+be controlled by the metastore. Managed tables will also have their data deleted automatically
+when a table is dropped.
+
## Parquet Files
[Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems.
@@ -492,16 +819,16 @@ Using the data from the above example:
{% highlight scala %}
// sqlContext from the previous example is used in this example.
-// createSchemaRDD is used to implicitly convert an RDD to a SchemaRDD.
-import sqlContext.createSchemaRDD
+// This is used to implicitly convert an RDD to a DataFrame.
+import sqlContext.implicits._
val people: RDD[Person] = ... // An RDD of case class objects, from the previous example.
-// The RDD is implicitly converted to a SchemaRDD by createSchemaRDD, allowing it to be stored using Parquet.
+// The RDD is implicitly converted to a DataFrame by implicits, allowing it to be stored using Parquet.
people.saveAsParquetFile("people.parquet")
// Read in the parquet file created above. Parquet files are self-describing so the schema is preserved.
-// The result of loading a Parquet file is also a SchemaRDD.
+// The result of loading a Parquet file is also a DataFrame.
val parquetFile = sqlContext.parquetFile("people.parquet")
//Parquet files can also be registered as tables and then used in SQL statements.
@@ -517,18 +844,18 @@ teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
{% highlight java %}
// sqlContext from the previous example is used in this example.
-JavaSchemaRDD schemaPeople = ... // The JavaSchemaRDD from the previous example.
+DataFrame schemaPeople = ... // The DataFrame from the previous example.
-// JavaSchemaRDDs can be saved as Parquet files, maintaining the schema information.
+// DataFrames can be saved as Parquet files, maintaining the schema information.
schemaPeople.saveAsParquetFile("people.parquet");
// Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved.
-// The result of loading a parquet file is also a JavaSchemaRDD.
-JavaSchemaRDD parquetFile = sqlContext.parquetFile("people.parquet");
+// The result of loading a parquet file is also a DataFrame.
+DataFrame parquetFile = sqlContext.parquetFile("people.parquet");
//Parquet files can also be registered as tables and then used in SQL statements.
parquetFile.registerTempTable("parquetFile");
-JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19");
+DataFrame teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19");
List teenagerNames = teenagers.map(new Function() {
public String call(Row row) {
return "Name: " + row.getString(0);
@@ -543,13 +870,13 @@ List teenagerNames = teenagers.map(new Function() {
{% highlight python %}
# sqlContext from the previous example is used in this example.
-schemaPeople # The SchemaRDD from the previous example.
+schemaPeople # The DataFrame from the previous example.
-# SchemaRDDs can be saved as Parquet files, maintaining the schema information.
+# DataFrames can be saved as Parquet files, maintaining the schema information.
schemaPeople.saveAsParquetFile("people.parquet")
# Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved.
-# The result of loading a parquet file is also a SchemaRDD.
+# The result of loading a parquet file is also a DataFrame.
parquetFile = sqlContext.parquetFile("people.parquet")
# Parquet files can also be registered as tables and then used in SQL statements.
@@ -562,11 +889,150 @@ for teenName in teenNames.collect():
+
+
+{% highlight sql %}
+
+CREATE TEMPORARY TABLE parquetTable
+USING org.apache.spark.sql.parquet
+OPTIONS (
+ path "examples/src/main/resources/people.parquet"
+)
+
+SELECT * FROM parquetTable
+
+{% endhighlight %}
+
+
+
+
+
+### Partition discovery
+
+Table partitioning is a common optimization approach used in systems like Hive. In a partitioned
+table, data are usually stored in different directories, with partitioning column values encoded in
+the path of each partition directory. The Parquet data source is now able to discover and infer
+partitioning information automatically. For exmaple, we can store all our previously used
+population data into a partitioned table using the following directory structure, with two extra
+columns, `gender` and `country` as partitioning columns:
+
+{% highlight text %}
+
+path
+└── to
+ └── table
+ ├── gender=male
+ │  ├── ...
+ │  │
+ │  ├── country=US
+ │  │  └── data.parquet
+ │  ├── country=CN
+ │  │  └── data.parquet
+ │  └── ...
+ └── gender=female
+   ├── ...
+   │
+   ├── country=US
+   │  └── data.parquet
+   ├── country=CN
+   │  └── data.parquet
+   └── ...
+
+{% endhighlight %}
+
+By passing `path/to/table` to either `SQLContext.parquetFile` or `SQLContext.load`, Spark SQL will
+automatically extract the partitioning information from the paths. Now the schema of the returned
+DataFrame becomes:
+
+{% highlight text %}
+
+root
+|-- name: string (nullable = true)
+|-- age: long (nullable = true)
+|-- gender: string (nullable = true)
+|-- country: string (nullable = true)
+
+{% endhighlight %}
+
+Notice that the data types of the partitioning columns are automatically inferred. Currently,
+numeric data types and string type are supported.
+
+### Schema merging
+
+Like ProtocolBuffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with
+a simple schema, and gradually add more columns to the schema as needed. In this way, users may end
+up with multiple Parquet files with different but mutually compatible schemas. The Parquet data
+source is now able to automatically detect this case and merge schemas of all these files.
+
+
+
+
+
+{% highlight scala %}
+// sqlContext from the previous example is used in this example.
+// This is used to implicitly convert an RDD to a DataFrame.
+import sqlContext.implicits._
+
+// Create a simple DataFrame, stored into a partition directory
+val df1 = sparkContext.makeRDD(1 to 5).map(i => (i, i * 2)).toDF("single", "double")
+df1.saveAsParquetFile("data/test_table/key=1")
+
+// Create another DataFrame in a new partition directory,
+// adding a new column and dropping an existing column
+val df2 = sparkContext.makeRDD(6 to 10).map(i => (i, i * 3)).toDF("single", "triple")
+df2.saveAsParquetFile("data/test_table/key=2")
+
+// Read the partitioned table
+val df3 = sqlContext.parquetFile("data/test_table")
+df3.printSchema()
+
+// The final schema consists of all 3 columns in the Parquet files together
+// with the partiioning column appeared in the partition directory paths.
+// root
+// |-- single: int (nullable = true)
+// |-- double: int (nullable = true)
+// |-- triple: int (nullable = true)
+// |-- key : int (nullable = true)
+{% endhighlight %}
+
+
+
+
+
+{% highlight python %}
+# sqlContext from the previous example is used in this example.
+
+# Create a simple DataFrame, stored into a partition directory
+df1 = sqlContext.createDataFrame(sc.parallelize(range(1, 6))\
+ .map(lambda i: Row(single=i, double=i * 2)))
+df1.save("data/test_table/key=1", "parquet")
+
+# Create another DataFrame in a new partition directory,
+# adding a new column and dropping an existing column
+df2 = sqlContext.createDataFrame(sc.parallelize(range(6, 11))
+ .map(lambda i: Row(single=i, triple=i * 3)))
+df2.save("data/test_table/key=2", "parquet")
+
+# Read the partitioned table
+df3 = sqlContext.parquetFile("data/test_table")
+df3.printSchema()
+
+# The final schema consists of all 3 columns in the Parquet files together
+# with the partiioning column appeared in the partition directory paths.
+# root
+# |-- single: int (nullable = true)
+# |-- double: int (nullable = true)
+# |-- triple: int (nullable = true)
+# |-- key : int (nullable = true)
+{% endhighlight %}
+
+
+
### Configuration
-Configuration of Parquet can be done using the `setConf` method on SQLContext or by running
+Configuration of Parquet can be done using the `setConf` method on `SQLContext` or by running
`SET key=value` commands using SQL.
@@ -580,6 +1046,15 @@ Configuration of Parquet can be done using the `setConf` method on SQLContext or
flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems.
+
+ spark.sql.parquet.int96AsTimestamp
+ true
+
+ Some Parquet-producing systems, in particular Impala, store Timestamp into INT96. Spark would also
+ store Timestamp as INT96 because we need to avoid precision lost of the nanoseconds field. This
+ flag tells Spark SQL to interpret INT96 data as a timestamp to provide compatibility with these systems.
+
+
spark.sql.parquet.cacheMetadata
true
@@ -619,8 +1094,8 @@ Configuration of Parquet can be done using the `setConf` method on SQLContext or
-Spark SQL can automatically infer the schema of a JSON dataset and load it as a SchemaRDD.
-This conversion can be done using one of two methods in a SQLContext:
+Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame.
+This conversion can be done using one of two methods in a `SQLContext`:
* `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object.
* `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object.
@@ -636,7 +1111,7 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc)
// A JSON dataset is pointed to by path.
// The path can be either a single text file or a directory storing text files.
val path = "examples/src/main/resources/people.json"
-// Create a SchemaRDD from the file(s) pointed to by path
+// Create a DataFrame from the file(s) pointed to by path
val people = sqlContext.jsonFile(path)
// The inferred schema can be visualized using the printSchema() method.
@@ -645,13 +1120,13 @@ people.printSchema()
// |-- age: integer (nullable = true)
// |-- name: string (nullable = true)
-// Register this SchemaRDD as a table.
+// Register this DataFrame as a table.
people.registerTempTable("people")
// SQL statements can be run by using the sql methods provided by sqlContext.
val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19")
-// Alternatively, a SchemaRDD can be created for a JSON dataset represented by
+// Alternatively, a DataFrame can be created for a JSON dataset represented by
// an RDD[String] storing one JSON object per string.
val anotherPeopleRDD = sc.parallelize(
"""{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil)
@@ -661,8 +1136,8 @@ val anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD)
-Spark SQL can automatically infer the schema of a JSON dataset and load it as a JavaSchemaRDD.
-This conversion can be done using one of two methods in a JavaSQLContext :
+Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame.
+This conversion can be done using one of two methods in a `SQLContext` :
* `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object.
* `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object.
@@ -673,13 +1148,13 @@ a regular multi-line JSON file will most often fail.
{% highlight java %}
// sc is an existing JavaSparkContext.
-JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc);
+SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
// A JSON dataset is pointed to by path.
// The path can be either a single text file or a directory storing text files.
String path = "examples/src/main/resources/people.json";
-// Create a JavaSchemaRDD from the file(s) pointed to by path
-JavaSchemaRDD people = sqlContext.jsonFile(path);
+// Create a DataFrame from the file(s) pointed to by path
+DataFrame people = sqlContext.jsonFile(path);
// The inferred schema can be visualized using the printSchema() method.
people.printSchema();
@@ -687,24 +1162,24 @@ people.printSchema();
// |-- age: integer (nullable = true)
// |-- name: string (nullable = true)
-// Register this JavaSchemaRDD as a table.
+// Register this DataFrame as a table.
people.registerTempTable("people");
// SQL statements can be run by using the sql methods provided by sqlContext.
-JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19");
+DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19");
-// Alternatively, a JavaSchemaRDD can be created for a JSON dataset represented by
+// Alternatively, a DataFrame can be created for a JSON dataset represented by
// an RDD[String] storing one JSON object per string.
List jsonData = Arrays.asList(
"{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}");
JavaRDD anotherPeopleRDD = sc.parallelize(jsonData);
-JavaSchemaRDD anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD);
+DataFrame anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD);
{% endhighlight %}
-Spark SQL can automatically infer the schema of a JSON dataset and load it as a SchemaRDD.
-This conversion can be done using one of two methods in a SQLContext:
+Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame.
+This conversion can be done using one of two methods in a `SQLContext`:
* `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object.
* `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object.
@@ -721,7 +1196,7 @@ sqlContext = SQLContext(sc)
# A JSON dataset is pointed to by path.
# The path can be either a single text file or a directory storing text files.
path = "examples/src/main/resources/people.json"
-# Create a SchemaRDD from the file(s) pointed to by path
+# Create a DataFrame from the file(s) pointed to by path
people = sqlContext.jsonFile(path)
# The inferred schema can be visualized using the printSchema() method.
@@ -730,13 +1205,13 @@ people.printSchema()
# |-- age: integer (nullable = true)
# |-- name: string (nullable = true)
-# Register this SchemaRDD as a table.
+# Register this DataFrame as a table.
people.registerTempTable("people")
-# SQL statements can be run by using the sql methods provided by sqlContext.
+# SQL statements can be run by using the sql methods provided by `sqlContext`.
teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19")
-# Alternatively, a SchemaRDD can be created for a JSON dataset represented by
+# Alternatively, a DataFrame can be created for a JSON dataset represented by
# an RDD[String] storing one JSON object per string.
anotherPeopleRDD = sc.parallelize([
'{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}'])
@@ -744,6 +1219,22 @@ anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD)
{% endhighlight %}
+
+
+{% highlight sql %}
+
+CREATE TEMPORARY TABLE jsonTable
+USING org.apache.spark.sql.json
+OPTIONS (
+ path "examples/src/main/resources/people.json"
+)
+
+SELECT * FROM jsonTable
+
+{% endhighlight %}
+
+
+
## Hive Tables
@@ -763,7 +1254,7 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`.
When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and
adds support for finding tables in the MetaStore and writing queries using HiveQL. Users who do
-not have an existing Hive deployment can still create a HiveContext. When not configured by the
+not have an existing Hive deployment can still create a `HiveContext`. When not configured by the
hive-site.xml, the context automatically creates `metastore_db` and `warehouse` in the current
directory.
@@ -782,14 +1273,14 @@ sqlContext.sql("FROM src SELECT key, value").collect().foreach(println)
-When working with Hive one must construct a `JavaHiveContext`, which inherits from `JavaSQLContext`, and
+When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and
adds support for finding tables in the MetaStore and writing queries using HiveQL. In addition to
-the `sql` method a `JavaHiveContext` also provides an `hql` methods, which allows queries to be
+the `sql` method a `HiveContext` also provides an `hql` methods, which allows queries to be
expressed in HiveQL.
{% highlight java %}
// sc is an existing JavaSparkContext.
-JavaHiveContext sqlContext = new org.apache.spark.sql.hive.api.java.HiveContext(sc);
+HiveContext sqlContext = new org.apache.spark.sql.hive.HiveContext(sc);
sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)");
sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src");
@@ -824,6 +1315,121 @@ results = sqlContext.sql("FROM src SELECT key, value").collect()
+## JDBC To Other Databases
+
+Spark SQL also includes a data source that can read data from other databases using JDBC. This
+functionality should be preferred over using [JdbcRDD](api/scala/index.html#org.apache.spark.rdd.JdbcRDD).
+This is because the results are returned
+as a DataFrame and they can easily be processed in Spark SQL or joined with other data sources.
+The JDBC data source is also easier to use from Java or Python as it does not require the user to
+provide a ClassTag.
+(Note that this is different than the Spark SQL JDBC server, which allows other applications to
+run queries using Spark SQL).
+
+To get started you will need to include the JDBC driver for you particular database on the
+spark classpath. For example, to connect to postgres from the Spark Shell you would run the
+following command:
+
+{% highlight bash %}
+SPARK_CLASSPATH=postgresql-9.3-1102-jdbc41.jar bin/spark-shell
+{% endhighlight %}
+
+Tables from the remote database can be loaded as a DataFrame or Spark SQL Temporary table using
+the Data Sources API. The following options are supported:
+
+
+ Property Name Meaning
+
+ url
+
+ The JDBC URL to connect to.
+
+
+
+ dbtable
+
+ The JDBC table that should be read. Note that anything that is valid in a `FROM` clause of
+ a SQL query can be used. For example, instead of a full table you could also use a
+ subquery in parentheses.
+
+
+
+
+ driver
+
+ The class name of the JDBC driver needed to connect to this URL. This class with be loaded
+ on the master and workers before running an JDBC commands to allow the driver to
+ register itself with the JDBC subsystem.
+
+
+
+ partitionColumn, lowerBound, upperBound, numPartitions
+
+ These options must all be specified if any of them is specified. They describe how to
+ partition the table when reading in parallel from multiple workers.
+ partitionColumn
must be a numeric column from the table in question.
+
+
+
+
+
+
+
+
+{% highlight scala %}
+val jdbcDF = sqlContext.load("jdbc", Map(
+ "url" -> "jdbc:postgresql:dbserver",
+ "dbtable" -> "schema.tablename"))
+{% endhighlight %}
+
+
+
+
+
+{% highlight java %}
+
+Map options = new HashMap();
+options.put("url", "jdbc:postgresql:dbserver");
+options.put("dbtable", "schema.tablename");
+
+DataFrame jdbcDF = sqlContext.load("jdbc", options)
+{% endhighlight %}
+
+
+
+
+
+
+{% highlight python %}
+
+df = sqlContext.load("jdbc", url="jdbc:postgresql:dbserver", dbtable="schema.tablename")
+
+{% endhighlight %}
+
+
+
+
+
+{% highlight sql %}
+
+CREATE TEMPORARY TABLE jdbcTable
+USING org.apache.spark.sql.jdbc
+OPTIONS (
+ url "jdbc:postgresql:dbserver",
+ dbtable "schema.tablename"
+)
+
+{% endhighlight %}
+
+
+
+
+## Troubleshooting
+
+ * The JDBC driver class must be visible to the primordial class loader on the client session and on all executors. This is because Java's DriverManager class does a security check that results in it ignoring all drivers not visible to the primordial class loader when one goes to open a connection. One convenient way to do this is to modify compute_classpath.sh on all worker nodes to include your driver JARs.
+ * Some databases, such as H2, convert all names to upper case. You'll need to use upper case to refer to those names in Spark SQL.
+
+
# Performance Tuning
For some workloads it is possible to improve performance by either caching data in memory, or by
@@ -831,11 +1437,11 @@ turning on some experimental options.
## Caching Data In Memory
-Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")` or `schemaRDD.cache()`.
+Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")` or `dataFrame.cache()`.
Then Spark SQL will scan only required columns and will automatically tune compression to minimize
memory usage and GC pressure. You can call `sqlContext.uncacheTable("tableName")` to remove the table from memory.
-Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running
+Configuration of in-memory caching can be done using the `setConf` method on `SQLContext` or by running
`SET key=value` commands using SQL.
@@ -894,15 +1500,14 @@ that these options will be deprecated in future release as more optimizations ar
-# Other SQL Interfaces
+# Distributed SQL Engine
-Spark SQL also supports interfaces for running SQL queries directly without the need to write any
-code.
+Spark SQL can also act as a distributed query engine using its JDBC/ODBC or command-line interface. In this mode, end-users or applications can interact with Spark SQL directly to run SQL queries, without the need to write any code.
## Running the Thrift JDBC/ODBC server
The Thrift JDBC/ODBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2)
-in Hive 0.12. You can test the JDBC server with the beeline script that comes with either Spark or Hive 0.12.
+in Hive 0.13. You can test the JDBC server with the beeline script that comes with either Spark or Hive 0.13.
To start the JDBC/ODBC server, run the following in the Spark directory:
@@ -947,10 +1552,10 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`.
You may also use the beeline script that comes with Hive.
-Thrift JDBC server also supports sending thrift RPC messages over HTTP transport.
-Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`:
+Thrift JDBC server also supports sending thrift RPC messages over HTTP transport.
+Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`:
- hive.server2.transport.mode - Set this to value: http
+ hive.server2.transport.mode - Set this to value: http
hive.server2.thrift.http.port - HTTP port number fo listen on; default is 10001
hive.server2.http.endpoint - HTTP endpoint; default is cliservice
@@ -972,7 +1577,88 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`.
You may run `./bin/spark-sql --help` for a complete list of all available
options.
-# Compatibility with Other Systems
+# Migration Guide
+
+## Upgrading from Spark SQL 1.0-1.2 to 1.3
+
+In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the
+available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other
+releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked
+as unstable (i.e., DeveloperAPI or Experimental).
+
+#### Rename of SchemaRDD to DataFrame
+
+The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has
+been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD
+directly, but instead provide most of the functionality that RDDs provide though their own
+implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method.
+
+In Scala there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for
+some use cases. It is still recommended that users update their code to use `DataFrame` instead.
+Java and Python users will need to update their code.
+
+#### Unification of the Java and Scala APIs
+
+Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`)
+that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users
+of either language should use `SQLContext` and `DataFrame`. In general theses classes try to
+use types that are usable from both languages (i.e. `Array` instead of language specific collections).
+In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading
+is used instead.
+
+Additionally the Java specific types API has been removed. Users of both Scala and Java should
+use the classes present in `org.apache.spark.sql.types` to describe schema programmatically.
+
+
+#### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only)
+
+Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought
+all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit
+conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`.
+Users should now write `import sqlContext.implicits._`.
+
+Additionally, the implicit conversions now only augment RDDs that are composed of `Product`s (i.e.,
+case classes or tuples) with a method `toDF`, instead of applying automatically.
+
+When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import
+`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used:
+`import org.apache.spark.sql.functions._`.
+
+#### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only)
+
+Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users
+should instead import the classes in `org.apache.spark.sql.types`
+
+#### UDF Registration Moved to `sqlContext.udf` (Java & Scala)
+
+Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been
+moved into the udf object in `SQLContext`.
+
+
+
+{% highlight java %}
+
+sqlCtx.udf.register("strLen", (s: String) => s.length())
+
+{% endhighlight %}
+
+
+
+{% highlight java %}
+
+sqlCtx.udf().register("strLen", (String s) -> { s.length(); });
+
+{% endhighlight %}
+
+
+
+
+Python UDF registration is unchanged.
+
+#### Python DataTypes No Longer Singletons
+
+When using DataTypes in Python you will need to construct them (i.e. `StringType()`) instead of
+referencing a singleton.
## Migration Guide for Shark User
@@ -1092,15 +1778,10 @@ in Hive deployments.
* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL
doesn't support buckets yet.
-**Esoteric Hive Features**
-* Tables with partitions using different input formats: In Spark SQL, all table partitions need to
- have the same input format.
-* Non-equi outer join: For the uncommon use case of using outer joins with non-equi join conditions
- (e.g. condition "`key < 10`"), Spark SQL will output wrong result for the `NULL` tuple.
-* `UNION` type and `DATE` type
+**Esoteric Hive Features**
+* `UNION` type
* Unique join
-* Single query multi insert
* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at
the moment and only supports populating the sizeInBytes field of the hive metastore.
@@ -1116,9 +1797,6 @@ less important due to Spark SQL's in-memory computational model. Others are slot
releases of Spark SQL.
* Block level bitmap indexes and virtual columns (used to build indexes)
-* Automatically convert a join to map join: For joining a large table with multiple small tables,
- Hive automatically converts the join into a map join. We are adding this auto conversion in the
- next release.
* Automatically determine the number of reducers for joins and groupbys: Currently in Spark SQL, you
need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`".
* Meta-data only query: For queries that can be answered by using only meta data, Spark SQL still
@@ -1129,33 +1807,10 @@ releases of Spark SQL.
Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS
metadata. Spark SQL does not support that.
-# Writing Language-Integrated Relational Queries
-**Language-Integrated queries are experimental and currently only supported in Scala.**
+# Data Types
-Spark SQL also supports a domain specific language for writing queries. Once again,
-using the data from the above examples:
-
-{% highlight scala %}
-// sc is an existing SparkContext.
-val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-// Importing the SQL context gives access to all the public SQL functions and implicit conversions.
-import sqlContext._
-val people: RDD[Person] = ... // An RDD of case class objects, from the first example.
-
-// The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19'
-val teenagers = people.where('age >= 10).where('age <= 19).select('name)
-teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
-{% endhighlight %}
-
-The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers
-prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are
-evaluated by the SQL execution engine. A full list of the functions supported can be found in the
-[ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD).
-
-
-
-# Spark SQL DataType Reference
+Spark SQL and DataFrames support the following data types:
* Numeric types
- `ByteType`: Represents 1-byte signed integer numbers.
@@ -1198,10 +1853,10 @@ evaluated by the SQL execution engine. A full list of the functions supported c
-All data types of Spark SQL are located in the package `org.apache.spark.sql`.
+All data types of Spark SQL are located in the package `org.apache.spark.sql.types`.
You can access them by doing
{% highlight scala %}
-import org.apache.spark.sql._
+import org.apache.spark.sql.types._
{% endhighlight %}
@@ -1253,7 +1908,7 @@ import org.apache.spark.sql._
DecimalType
- scala.math.BigDecimal
+ java.math.BigDecimal
DecimalType
@@ -1447,7 +2102,7 @@ please use factory methods provided in
StructType
- org.apache.spark.sql.api.java.Row
+ org.apache.spark.sql.Row
DataTypes.createStructType(fields)
Note: fields is a List or an array of StructFields.
@@ -1468,10 +2123,10 @@ please use factory methods provided in
-All data types of Spark SQL are located in the package of `pyspark.sql`.
+All data types of Spark SQL are located in the package of `pyspark.sql.types`.
You can access them by doing
{% highlight python %}
-from pyspark.sql import *
+from pyspark.sql.types import *
{% endhighlight %}
diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md
index ac01dd3d8019a..c8ab146bcae0a 100644
--- a/docs/streaming-flume-integration.md
+++ b/docs/streaming-flume-integration.md
@@ -5,6 +5,8 @@ title: Spark Streaming + Flume Integration Guide
[Apache Flume](https://flume.apache.org/) is a distributed, reliable, and available service for efficiently collecting, aggregating, and moving large amounts of log data. Here we explain how to configure Flume and Spark Streaming to receive data from Flume. There are two approaches to this.
+Python API Flume is not yet available in the Python API.
+
## Approach 1: Flume-style Push-based Approach
Flume is designed to push data between Flume agents. In this approach, Spark Streaming essentially sets up a receiver that acts an Avro agent for Flume, to which Flume can push the data. Here are the configuration steps.
@@ -64,7 +66,7 @@ configuring Flume agents.
3. **Deploying:** Package `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide).
-## Approach 2 (Experimental): Pull-based Approach using a Custom Sink
+## Approach 2: Pull-based Approach using a Custom Sink
Instead of Flume pushing data directly to Spark Streaming, this approach runs a custom Flume sink that allows the following.
- Flume pushes data into the sink, and the data stays buffered.
diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md
index 77c0abbbacbd0..64714f0b799fc 100644
--- a/docs/streaming-kafka-integration.md
+++ b/docs/streaming-kafka-integration.md
@@ -2,58 +2,155 @@
layout: global
title: Spark Streaming + Kafka Integration Guide
---
-[Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka.
+[Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka. There are two approaches to this - the old approach using Receivers and Kafka's high-level API, and a new experimental approach (introduced in Spark 1.3) without using Receivers. They have different programming models, performance characteristics, and semantics guarantees, so read on for more details.
-1. **Linking:** In your SBT/Maven project definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information).
+## Approach 1: Receiver-based Approach
+This approach uses a Receiver to receive the data. The Received is implemented using the Kafka high-level consumer API. As with all receivers, the data received from Kafka through a Receiver is stored in Spark executors, and then jobs launched by Spark Streaming processes the data.
+
+However, under default configuration, this approach can lose data under failures (see [receiver reliability](streaming-programming-guide.html#receiver-reliability). To ensure zero-data loss, you have to additionally enable Write Ahead Logs in Spark Streaming. To ensure zero data loss, enable the Write Ahead Logs (introduced in Spark 1.2). This synchronously saves all the received Kafka data into write ahead logs on a distributed file system (e.g HDFS), so that all the data can be recovered on failure. See [Deploying section](streaming-programming-guide.html#deploying-applications) in the streaming programming guide for more details on Write Ahead Logs.
+
+Next, we discuss how to use this approach in your streaming application.
+
+1. **Linking:** For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information).
groupId = org.apache.spark
artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}
version = {{site.SPARK_VERSION_SHORT}}
-2. **Programming:** In the streaming application code, import `KafkaUtils` and create input DStream as follows.
+ For Python applications, you will have to add this above library and its dependencies when deploying your application. See the *Deploying* subsection below.
+
+2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows.
import org.apache.spark.streaming.kafka._
- val kafkaStream = KafkaUtils.createStream(
- streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume])
+ val kafkaStream = KafkaUtils.createStream(streamingContext,
+ [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume])
- See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$)
+ You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$)
and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala).
import org.apache.spark.streaming.kafka.*;
- JavaPairReceiverInputDStream kafkaStream = KafkaUtils.createStream(
- streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume]);
+ JavaPairReceiverInputDStream kafkaStream =
+ KafkaUtils.createStream(streamingContext,
+ [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]);
- See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html)
+ You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html)
and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java).
+
+
+
+ from pyspark.streaming.kafka import KafkaUtils
+
+ kafkaStream = KafkaUtils.createStream(streamingContext, \
+ [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume])
+
+ By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils)
+ and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/kafka_wordcount.py).
- *Points to remember:*
+ **Points to remember:**
- Topic partitions in Kafka does not correlate to partitions of RDDs generated in Spark Streaming. So increasing the number of topic-specific partitions in the `KafkaUtils.createStream()` only increases the number of threads using which topics that are consumed within a single receiver. It does not increase the parallelism of Spark in processing the data. Refer to the main document for more information on that.
- Multiple Kafka input DStreams can be created with different groups and topics for parallel receiving of data using multiple receivers.
-3. **Deploying:** Package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide).
-
-Note that the Kafka receiver used by default is an
-[*unreliable* receiver](streaming-programming-guide.html#receiver-reliability) section in the
-programming guide). In Spark 1.2, we have added an experimental *reliable* Kafka receiver that
-provides stronger
-[fault-tolerance guarantees](streaming-programming-guide.html#fault-tolerance-semantics) of zero
-data loss on failures. This receiver is automatically used when the write ahead log
-(also introduced in Spark 1.2) is enabled
-(see [Deployment](#deploying-applications.html) section in the programming guide). This
-may reduce the receiving throughput of individual Kafka receivers compared to the unreliable
-receivers, but this can be corrected by running
-[more receivers in parallel](streaming-programming-guide.html#level-of-parallelism-in-data-receiving)
-to increase aggregate throughput. Additionally, it is recommended that the replication of the
-received data within Spark be disabled when the write ahead log is enabled as the log is already stored
-in a replicated storage system. This can be done by setting the storage level for the input
-stream to `StorageLevel.MEMORY_AND_DISK_SER` (that is, use
+ - If you have enabled Write Ahead Logs with a replicated file system like HDFS, the received data is already being replicated in the log. Hence, the storage level in storage level for the input stream to `StorageLevel.MEMORY_AND_DISK_SER` (that is, use
`KafkaUtils.createStream(..., StorageLevel.MEMORY_AND_DISK_SER)`).
+
+3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications.
+
+ For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide).
+
+ For Python applications which lack SBT/Maven project management, `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is,
+
+ ./bin/spark-submit --packages org.apache.spark:spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ...
+
+ Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-kafka-assembly` from the
+ [Maven repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-kafka-assembly_2.10%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`.
+
+## Approach 2: Direct Approach (No Receivers)
+This is a new receiver-less "direct" approach has been introduced in Spark 1.3 to ensure stronger end-to-end guarantees. Instead of using receivers to receive data, this approach periodically queries Kafka for the latest offsets in each topic+partition, and accordingly defines the offset ranges to process in each batch. When the jobs to process the data are launched, Kafka's simple consumer API is used to read the defined ranges of offsets from Kafka (similar to read files from a file system). Note that this is an experimental feature in Spark 1.3 and is only available in the Scala and Java API.
+
+This approach has the following advantages over the received-based approach (i.e. Approach 1).
+
+- *Simplified Parallelism:* No need to create multiple input Kafka streams and union-ing them. With `directStream`, Spark Streaming will create as many RDD partitions as there is Kafka partitions to consume, which will all read data from Kafka in parallel. So there is one-to-one mapping between Kafka and RDD partitions, which is easier to understand and tune.
+
+- *Efficiency:* Achieving zero-data loss in the first approach required the data to be stored in a Write Ahead Log, which further replicated the data. This is actually inefficient as the data effectively gets replicated twice - once by Kafka, and a second time by the Write Ahead Log. This second approach eliminate the problem as there is no receiver, and hence no need for Write Ahead Logs.
+
+- *Exactly-once semantics:* The first approach uses Kafka's high level API to store consumed offsets in Zookeeper. This is traditionally the way to consume data from Kafka. While this approach (in combination with write ahead logs) can ensure zero data loss (i.e. at-least once semantics), there is a small chance some records may get consumed twice under some failures. This occurs because of inconsistencies between data reliably received by Spark Streaming and offsets tracked by Zookeeper. Hence, in this second approach, we use simple Kafka API that does not use Zookeeper and offsets tracked only by Spark Streaming within its checkpoints. This eliminates inconsistencies between Spark Streaming and Zookeeper/Kafka, and so each record is received by Spark Streaming effectively exactly once despite failures.
+
+Note that one disadvantage of this approach is that it does not update offsets in Zookeeper, hence Zookeeper-based Kafka monitoring tools will not show progress. However, you can access the offsets processed by this approach in each batch and update Zookeeper yourself (see below).
+
+Next, we discuss how to use this approach in your streaming application.
+
+1. **Linking:** This approach is supported only in Scala/Java application. Link your SBT/Maven project with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information).
+
+ groupId = org.apache.spark
+ artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}
+ version = {{site.SPARK_VERSION_SHORT}}
+
+2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows.
+
+
+
+ import org.apache.spark.streaming.kafka._
+
+ val directKafkaStream = KafkaUtils.createDirectStream[
+ [key class], [value class], [key decoder class], [value decoder class] ](
+ streamingContext, [map of Kafka parameters], [set of topics to consume])
+
+ See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$)
+ and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala).
+
+
+ import org.apache.spark.streaming.kafka.*;
+
+ JavaPairReceiverInputDStream directKafkaStream =
+ KafkaUtils.createDirectStream(streamingContext,
+ [key class], [value class], [key decoder class], [value decoder class],
+ [map of Kafka parameters], [set of topics to consume]);
+
+ See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html)
+ and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java).
+
+
+
+
+ In the Kafka parameters, you must specify either `metadata.broker.list` or `bootstrap.servers`.
+ By default, it will start consuming from the latest offset of each Kafka partition. If you set configuration `auto.offset.reset` in Kafka parameters to `smallest`, then it will start consuming from the smallest offset.
+
+ You can also start consuming from any arbitrary offset using other variations of `KafkaUtils.createDirectStream`. Furthermore, if you want to access the Kafka offsets consumed in each batch, you can do the following.
+
+
+
+ directKafkaStream.foreachRDD { rdd =>
+ val offsetRanges = rdd.asInstanceOf[HasOffsetRanges]
+ // offsetRanges.length = # of Kafka partitions being consumed
+ ...
+ }
+
+
+ directKafkaStream.foreachRDD(
+ new Function, Void>() {
+ @Override
+ public Void call(JavaPairRDD rdd) throws IOException {
+ OffsetRange[] offsetRanges = ((HasOffsetRanges)rdd).offsetRanges
+ // offsetRanges.length = # of Kafka partitions being consumed
+ ...
+ return null;
+ }
+ }
+ );
+
+
+
+ You can use this to update Zookeeper yourself if you want Zookeeper-based Kafka monitoring tools to show progress of the streaming application.
+
+ Another thing to note is that since this approach does not use Receivers, the standard receiver-related (that is, [configurations](configuration.html) of the form `spark.streaming.receiver.*` ) will not apply to the input DStreams created by this approach (will apply to other input DStreams though). Instead, use the [configurations](configuration.html) `spark.streaming.kafka.*`. An important one is `spark.streaming.kafka.maxRatePerPartition` which is the maximum rate at which each Kafka partition will be read by this direct API.
+
+3. **Deploying:** Similar to the first approach, you can package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR and the launch the application using `spark-submit`. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation.
\ No newline at end of file
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index e37a2bb37b9a4..6d6229625f3f9 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -1,6 +1,8 @@
---
layout: global
-title: Spark Streaming Programming Guide
+displayTitle: Spark Streaming Programming Guide
+title: Spark Streaming
+description: Spark Streaming programming guide and tutorial for Spark SPARK_VERSION_SHORT
---
* This will become a table of contents (this text will be scraped).
@@ -187,15 +189,15 @@ Next, we want to count these words.
{% highlight java %}
// Count each word in each batch
-JavaPairDStream pairs = words.map(
+JavaPairDStream pairs = words.mapToPair(
new PairFunction() {
- @Override public Tuple2 call(String s) throws Exception {
+ @Override public Tuple2 call(String s) {
return new Tuple2(s, 1);
}
});
JavaPairDStream wordCounts = pairs.reduceByKey(
new Function2() {
- @Override public Integer call(Integer i1, Integer i2) throws Exception {
+ @Override public Integer call(Integer i1, Integer i2) {
return i1 + i2;
}
});
@@ -430,7 +432,7 @@ some of the common ones are as follows.
For an up-to-date list, please refer to the
-[Apache repository](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22)
+[Maven repository](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22)
for the full list of supported sources and artifacts.
***
@@ -660,8 +662,7 @@ methods for creating DStreams from files and Akka actors as input sources.
For simple text files, there is an easier method `streamingContext.textFileStream(dataDirectory)`. And file streams do not require running a receiver, hence does not require allocating cores.
- Python API As of Spark 1.2,
- `fileStream` is not available in the Python API, only `textFileStream` is available.
+ Python API `fileStream` is not available in the Python API, only `textFileStream` is available.
- **Streams based on Custom Actors:** DStreams can be created with data streams received through Akka
actors by using `streamingContext.actorStream(actorProps, actor-name)`. See the [Custom Receiver
@@ -680,8 +681,9 @@ for Java, and [StreamingContext](api/python/pyspark.streaming.html#pyspark.strea
### Advanced Sources
{:.no_toc}
-Python API As of Spark 1.2,
-these sources are not available in the Python API.
+
+Python API As of Spark 1.3,
+out of these sources, *only* Kafka is available in the Python API. We will add more advanced sources in the Python API in future.
This category of sources require interfacing with external non-Spark libraries, some of them with
complex dependencies (e.g., Kafka and Flume). Hence, to minimize issues related to version conflicts
@@ -721,6 +723,12 @@ and it in the classpath.
Some of these advanced sources are as follows.
+- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka 0.8.1.1. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details.
+
+- **Flume:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Flume 1.4.0. See the [Flume Integration Guide](streaming-flume-integration.html) for more details.
+
+- **Kinesis:** See the [Kinesis Integration Guide](streaming-kinesis-integration.html) for more details.
+
- **Twitter:** Spark Streaming's TwitterUtils uses Twitter4j 3.0.3 to get the public stream of tweets using
[Twitter's Streaming API](https://dev.twitter.com/docs/streaming-apis). Authentication information
can be provided by any of the [methods](http://twitter4j.org/en/configuration.html) supported by
@@ -730,17 +738,10 @@ Some of these advanced sources are as follows.
([TwitterPopularTags]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala)
and [TwitterAlgebirdCMS]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala)).
-- **Flume:** Spark Streaming {{site.SPARK_VERSION_SHORT}} can received data from Flume 1.4.0. See the [Flume Integration Guide](streaming-flume-integration.html) for more details.
-
-- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} can receive data from Kafka 0.8.0. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details.
-
-- **Kinesis:** See the [Kinesis Integration Guide](streaming-kinesis-integration.html) for more details.
-
### Custom Sources
{:.no_toc}
-Python API As of Spark 1.2,
-these sources are not available in the Python API.
+Python API This is not yet supported in Python.
Input DStreams can also be created out of custom data sources. All you have to do is implement an
user-defined **receiver** (see next section to understand what that is) that can receive data from
@@ -844,7 +845,7 @@ Some of the common ones are as follows.
-The last two transformations are worth highlighting again.
+A few of these transformations are worth discussing in more detail.
#### UpdateStateByKey Operation
{:.no_toc}
@@ -876,6 +877,12 @@ This is applied on a DStream containing words (say, the `pairs` DStream containi
val runningCounts = pairs.updateStateByKey[Int](updateFunction _)
{% endhighlight %}
+The update function will be called for each word, with `newValues` having a sequence of 1's (from
+the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete
+Scala code, take a look at the example
+[StatefulNetworkWordCount.scala]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache
+/spark/examples/streaming/StatefulNetworkWordCount.scala).
+
@@ -897,6 +904,12 @@ This is applied on a DStream containing words (say, the `pairs` DStream containi
JavaPairDStream runningCounts = pairs.updateStateByKey(updateFunction);
{% endhighlight %}
+The update function will be called for each word, with `newValues` having a sequence of 1's (from
+the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete
+Java code, take a look at the example
+[JavaStatefulNetworkWordCount.java]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming
+/JavaStatefulNetworkWordCount.java).
+
@@ -914,14 +927,14 @@ This is applied on a DStream containing words (say, the `pairs` DStream containi
runningCounts = pairs.updateStateByKey(updateFunction)
{% endhighlight %}
-
-
-
The update function will be called for each word, with `newValues` having a sequence of 1's (from
the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete
-Scala code, take a look at the example
+Python code, take a look at the example
[stateful_network_wordcount.py]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/stateful_network_wordcount.py).
+
+
+
Note that using `updateStateByKey` requires the checkpoint directory to be configured, which is
discussed in detail in the [checkpointing](#checkpointing) section.
@@ -983,7 +996,7 @@ In fact, you can also use [machine learning](mllib-guide.html) and
#### Window Operations
{:.no_toc}
-Finally, Spark Streaming also provides *windowed computations*, which allow you to apply
+Spark Streaming also provides *windowed computations*, which allow you to apply
transformations over a sliding window of data. This following figure illustrates this sliding
window.
@@ -1027,7 +1040,7 @@ val windowedWordCounts = pairs.reduceByKeyAndWindow((a:Int,b:Int) => (a + b), Se
{% highlight java %}
// Reduce function adding two integers, defined separately for clarity
Function2 reduceFunc = new Function2() {
- @Override public Integer call(Integer i1, Integer i2) throws Exception {
+ @Override public Integer call(Integer i1, Integer i2) {
return i1 + i2;
}
};
@@ -1106,6 +1119,100 @@ said two parameters - windowLength and slideInterval.
+#### Join Operations
+{:.no_toc}
+Finally, its worth highlighting how easily you can perform different kinds of joins in Spark Streaming.
+
+
+##### Stream-stream joins
+{:.no_toc}
+Streams can be very easily joined with other streams.
+
+
+
+{% highlight scala %}
+val stream1: DStream[String, String] = ...
+val stream2: DStream[String, String] = ...
+val joinedStream = stream1.join(stream2)
+{% endhighlight %}
+
+
+{% highlight java %}
+JavaPairDStream stream1 = ...
+JavaPairDStream stream2 = ...
+JavaPairDStream joinedStream = stream1.join(stream2);
+{% endhighlight %}
+
+
+{% highlight python %}
+stream1 = ...
+stream2 = ...
+joinedStream = stream1.join(stream2)
+{% endhighlight %}
+
+
+Here, in each batch interval, the RDD generated by `stream1` will be joined with the RDD generated by `stream2`. You can also do `leftOuterJoin`, `rightOuterJoin`, `fullOuterJoin`. Furthermore, it is often very useful to do joins over windows of the streams. That is pretty easy as well.
+
+
+
+{% highlight scala %}
+val windowedStream1 = stream1.window(Seconds(20))
+val windowedStream2 = stream2.window(Minutes(1))
+val joinedStream = windowedStream1.join(windowedStream2)
+{% endhighlight %}
+
+
+{% highlight java %}
+JavaPairDStream windowedStream1 = stream1.window(Durations.seconds(20));
+JavaPairDStream windowedStream2 = stream2.window(Durations.minutes(1));
+JavaPairDStream joinedStream = windowedStream1.join(windowedStream2);
+{% endhighlight %}
+
+
+{% highlight python %}
+windowedStream1 = stream1.window(20)
+windowedStream2 = stream2.window(60)
+joinedStream = windowedStream1.join(windowedStream2)
+{% endhighlight %}
+
+
+
+##### Stream-dataset joins
+{:.no_toc}
+This has already been shown earlier while explain `DStream.transform` operation. Here is yet another example of joining a windowed stream with a dataset.
+
+
+
+{% highlight scala %}
+val dataset: RDD[String, String] = ...
+val windowedStream = stream.window(Seconds(20))...
+val joinedStream = windowedStream.transform { rdd => rdd.join(dataset) }
+{% endhighlight %}
+
+
+{% highlight java %}
+JavaPairRDD dataset = ...
+JavaPairDStream windowedStream = stream.window(Durations.seconds(20));
+JavaPairDStream joinedStream = windowedStream.transform(
+ new Function>, JavaRDD>>() {
+ @Override
+ public JavaRDD> call(JavaRDD> rdd) {
+ return rdd.join(dataset);
+ }
+ }
+);
+{% endhighlight %}
+
+
+{% highlight python %}
+dataset = ... # some RDD
+windowedStream = stream.window(20)
+joinedStream = windowedStream.transform(lambda rdd: rdd.join(dataset))
+{% endhighlight %}
+
+
+
+In fact, you can also dynamically change the dataset you want to join against. The function provided to `transform` is evaluated every batch interval and therefore will use the current dataset that `dataset` reference points to.
The complete list of DStream transformations is available in the API documentation. For the Scala API,
see [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream)
@@ -1313,6 +1420,178 @@ Note that the connections in the pool should be lazily created on demand and tim
***
+## DataFrame and SQL Operations
+You can easily use [DataFrames and SQL](sql-programming-guide.html) operations on streaming data. You have to create a SQLContext using the SparkContext that the StreamingContext is using. Furthermore this has to done such that it can be restarted on driver failures. This is done by creating a lazily instantiated singleton instance of SQLContext. This is shown in the following example. It modifies the earlier [word count example](#a-quick-example) to generate word counts using DataFrames and SQL. Each RDD is converted to a DataFrame, registered as a temporary table and then queried using SQL.
+
+
+
+{% highlight scala %}
+
+/** Lazily instantiated singleton instance of SQLContext */
+object SQLContextSingleton {
+ @transient private var instance: SQLContext = null
+
+ // Instantiate SQLContext on demand
+ def getInstance(sparkContext: SparkContext): SQLContext = synchronized {
+ if (instance == null) {
+ instance = new SQLContext(sparkContext)
+ }
+ instance
+ }
+}
+
+...
+
+/** Case class for converting RDD to DataFrame */
+case class Row(word: String)
+
+...
+
+/** DataFrame operations inside your streaming program */
+
+val words: DStream[String] = ...
+
+words.foreachRDD { rdd =>
+
+ // Get the singleton instance of SQLContext
+ val sqlContext = SQLContextSingleton.getInstance(rdd.sparkContext)
+ import sqlContext.implicits._
+
+ // Convert RDD[String] to RDD[case class] to DataFrame
+ val wordsDataFrame = rdd.map(w => Row(w)).toDF()
+
+ // Register as table
+ wordsDataFrame.registerTempTable("words")
+
+ // Do word count on DataFrame using SQL and print it
+ val wordCountsDataFrame =
+ sqlContext.sql("select word, count(*) as total from words group by word")
+ wordCountsDataFrame.show()
+}
+
+{% endhighlight %}
+
+See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala).
+
+
+{% highlight java %}
+
+/** Lazily instantiated singleton instance of SQLContext */
+class JavaSQLContextSingleton {
+ static private transient SQLContext instance = null;
+ static public SQLContext getInstance(SparkContext sparkContext) {
+ if (instance == null) {
+ instance = new SQLContext(sparkContext);
+ }
+ return instance;
+ }
+}
+
+...
+
+/** Java Bean class for converting RDD to DataFrame */
+public class JavaRow implements java.io.Serializable {
+ private String word;
+
+ public String getWord() {
+ return word;
+ }
+
+ public void setWord(String word) {
+ this.word = word;
+ }
+}
+
+...
+
+/** DataFrame operations inside your streaming program */
+
+JavaDStream words = ...
+
+words.foreachRDD(
+ new Function2, Time, Void>() {
+ @Override
+ public Void call(JavaRDD rdd, Time time) {
+ SQLContext sqlContext = JavaSQLContextSingleton.getInstance(rdd.context());
+
+ // Convert RDD[String] to RDD[case class] to DataFrame
+ JavaRDD rowRDD = rdd.map(new Function() {
+ public JavaRow call(String word) {
+ JavaRow record = new JavaRow();
+ record.setWord(word);
+ return record;
+ }
+ });
+ DataFrame wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRow.class);
+
+ // Register as table
+ wordsDataFrame.registerTempTable("words");
+
+ // Do word count on table using SQL and print it
+ DataFrame wordCountsDataFrame =
+ sqlContext.sql("select word, count(*) as total from words group by word");
+ wordCountsDataFrame.show();
+ return null;
+ }
+ }
+);
+{% endhighlight %}
+
+See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java).
+
+
+{% highlight python %}
+
+# Lazily instantiated global instance of SQLContext
+def getSqlContextInstance(sparkContext):
+ if ('sqlContextSingletonInstance' not in globals()):
+ globals()['sqlContextSingletonInstance'] = SQLContext(sparkContext)
+ return globals()['sqlContextSingletonInstance']
+
+...
+
+# DataFrame operations inside your streaming program
+
+words = ... # DStream of strings
+
+def process(time, rdd):
+ print "========= %s =========" % str(time)
+ try:
+ # Get the singleton instance of SQLContext
+ sqlContext = getSqlContextInstance(rdd.context)
+
+ # Convert RDD[String] to RDD[Row] to DataFrame
+ rowRdd = rdd.map(lambda w: Row(word=w))
+ wordsDataFrame = sqlContext.createDataFrame(rowRdd)
+
+ # Register as table
+ wordsDataFrame.registerTempTable("words")
+
+ # Do word count on table using SQL and print it
+ wordCountsDataFrame = sqlContext.sql("select word, count(*) as total from words group by word")
+ wordCountsDataFrame.show()
+ except:
+ pass
+
+words.foreachRDD(process)
+{% endhighlight %}
+
+See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/sql_network_wordcount.py).
+
+
+
+
+You can also run SQL queries on tables defined on streaming data from a different thread (that is, asynchronous to the running StreamingContext). Just make sure that you set the StreamingContext to remember sufficient amount of streaming data such that query can run. Otherwise the StreamingContext, which is unaware of the any asynchronous SQL queries, will delete off old streaming data before the query can complete. For example, if you want to query the last batch, but your query can take 5 minutes to run, then call `streamingContext.remember(Minutes(5))` (in Scala, or equivalent in other languages).
+
+See the [DataFrames and SQL](sql-programming-guide.html) guide to learn more about DataFrames.
+
+***
+
+## MLlib Operations
+You can also easily use machine learning algorithms provided by [MLlib](mllib-guide.html). First of all, there are streaming machine learning algorithms (e.g. (Streaming Linear Regression](mllib-linear-methods.html#streaming-linear-regression), [Streaming KMeans](mllib-clustering.html#streaming-k-means), etc.) which can simultaneously learn from the streaming data as well as apply the model on the streaming data. Beyond these, for a much larger class of machine learning algorithms, you can learn a learning model offline (i.e. using historical data) and then apply the model online on streaming data. See the [MLlib](mllib-guide.html) guide for more details.
+
+***
+
## Caching / Persistence
Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is,
using `persist()` method on a DStream will automatically persist every RDD of that DStream in
@@ -1566,9 +1845,8 @@ To run a Spark Streaming applications, you need to have the following.
+ *Mesos* - [Marathon](https://github.com/mesosphere/marathon) has been used to achieve this
with Mesos.
-
-- *[Experimental in Spark 1.2] Configuring write ahead logs* - In Spark 1.2,
- we have introduced a new experimental feature of write ahead logs for achieving strong
+- *[Since Spark 1.2] Configuring write ahead logs* - Since Spark 1.2,
+ we have introduced _write ahead logs_ for achieving strong
fault-tolerance guarantees. If enabled, all the data received from a receiver gets written into
a write ahead log in the configuration checkpoint directory. This prevents data loss on driver
recovery, thus ensuring zero data loss (discussed in detail in the
@@ -1654,7 +1932,7 @@ improve the performance of you application. At a high level, you need to conside
2. Setting the right batch size such that the batches of data can be processed as fast as they
are received (that is, data processing keeps up with the data ingestion).
-## Reducing the Processing Time of each Batch
+## Reducing the Batch Processing Times
There are a number of optimizations that can be done in Spark to minimize the processing time of
each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section
highlights some of the most important ones.
@@ -1726,16 +2004,15 @@ documentation), or set the `spark.default.parallelism`
### Data Serialization
{:.no_toc}
-The overhead of data serialization can be significant, especially when sub-second batch sizes are
- to be achieved. There are two aspects to it.
+The overheads of data serialization can be reduce by tuning the serialization formats. In case of streaming, there are two types of data that are being serialized.
-* **Serialization of RDD data in Spark**: Please refer to the detailed discussion on data
- serialization in the [Tuning Guide](tuning.html). However, note that unlike Spark, by default
- RDDs are persisted as serialized byte arrays to minimize pauses related to GC.
+* **Input data**: By default, the input data received through Receivers is stored in the executors' memory with [StorageLevel.MEMORY_AND_DISK_SER_2](api/scala/index.html#org.apache.spark.storage.StorageLevel$). That is, the data is serialized into bytes to reduce GC overheads, and replicated for tolerating executor failures. Also, the data is kept first in memory, and spilled over to disk only if the memory is unsufficient to hold all the input data necessary for the streaming computation. This serialization obviously has overheads -- the receiver must deserialize the received data and re-serialize it using Spark's serialization format.
-* **Serialization of input data**: To ingest external data into Spark, data received as bytes
- (say, from the network) needs to deserialized from bytes and re-serialized into Spark's
- serialization format. Hence, the deserialization overhead of input data may be a bottleneck.
+* **Persisted RDDs generated by Streaming Operations**: RDDs generated by streaming computations may be persisted in memory. For example, window operation persist data in memory as they would be processed multiple times. However, unlike Spark, by default RDDs are persisted with [StorageLevel.MEMORY_ONLY_SER](api/scala/index.html#org.apache.spark.storage.StorageLevel$) (i.e. serialized) to minimize GC overheads.
+
+In both cases, using Kryo serialization can reduce both CPU and memory overheads. See the [Spark Tuning Guide](tuning.html#data-serialization)) for more details. Consider registering custom classes, and disabling object reference tracking for Kryo (see Kryo-related configurations in the [Configuration Guide](configuration.html#compression-and-serialization)).
+
+In specific cases where the amount of data that needs to be retained for the streaming application is not large, it may be feasible to persist data (both types) as deserialized objects without incurring excessive GC overheads. For example, if you are using batch intervals of few seconds and no window operations, then you can try disabling serialization in persisted data by explicitly setting the storage level accordingly. This would reduce the CPU overheads due to serialization, potentially improving performance without too much GC overheads.
### Task Launching Overheads
{:.no_toc}
@@ -1755,7 +2032,7 @@ thus allowing sub-second batch size to be viable.
***
-## Setting the Right Batch Size
+## Setting the Right Batch Interval
For a Spark Streaming application running on a cluster to be stable, the system should be able to
process data as fast as it is being received. In other words, batches of data should be processed
as fast as they are being generated. Whether this is true for an application can be found by
@@ -1787,40 +2064,40 @@ temporary data rate increases maybe fine as long as the delay reduces back to a
## Memory Tuning
Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail
-in the [Tuning Guide](tuning.html). It is recommended that you read that. In this section,
-we highlight a few customizations that are strongly recommended to minimize GC related pauses
-in Spark Streaming applications and achieving more consistent batch processing times.
-
-* **Default persistence level of DStreams**: Unlike RDDs, the default persistence level of DStreams
-serializes the data in memory (that is,
-[StorageLevel.MEMORY_ONLY_SER](api/scala/index.html#org.apache.spark.storage.StorageLevel$) for
-DStream compared to
-[StorageLevel.MEMORY_ONLY](api/scala/index.html#org.apache.spark.storage.StorageLevel$) for RDDs).
-Even though keeping the data serialized incurs higher serialization/deserialization overheads,
-it significantly reduces GC pauses.
-
-* **Clearing persistent RDDs**: By default, all persistent RDDs generated by Spark Streaming will
- be cleared from memory based on Spark's built-in policy (LRU). If `spark.cleaner.ttl` is set,
- then persistent RDDs that are older than that value are periodically cleared. As mentioned
- [earlier](#operation), this needs to be careful set based on operations used in the Spark
- Streaming program. However, a smarter unpersisting of RDDs can be enabled by setting the
- [configuration property](configuration.html#spark-properties) `spark.streaming.unpersist` to
- `true`. This makes the system to figure out which RDDs are not necessary to be kept around and
- unpersists them. This is likely to reduce
- the RDD memory usage of Spark, potentially improving GC behavior as well.
-
-* **Concurrent garbage collector**: Using the concurrent mark-and-sweep GC further
-minimizes the variability of GC pauses. Even though concurrent GC is known to reduce the
+in the [Tuning Guide](tuning.html#memory-tuning). It is strongly recommended that you read that. In this section, we discuss a few tuning parameters specifically in the context of Spark Streaming applications.
+
+The amount of cluster memory required by a Spark Streaming application depends heavily on the type of transformations used. For example, if you want to use a window operation on last 10 minutes of data, then your cluster should have sufficient memory to hold 10 minutes of worth of data in memory. Or if you want to use `updateStateByKey` with a large number of keys, then the necessary memory will be high. On the contrary, if you want to do a simple map-filter-store operation, then necessary memory will be low.
+
+In general, since the data received through receivers are stored with StorageLevel.MEMORY_AND_DISK_SER_2, the data that does not fit in memory will spill over to the disk. This may reduce the performance of the streaming application, and hence it is advised to provide sufficient memory as required by your streaming application. Its best to try and see the memory usage on a small scale and estimate accordingly.
+
+Another aspect of memory tuning is garbage collection. For a streaming application that require low latency, it is undesirable to have large pauses caused by JVM Garbage Collection.
+
+There are a few parameters that can help you tune the memory usage and GC overheads.
+
+* **Persistence Level of DStreams**: As mentioned earlier in the [Data Serialization](#data-serialization) section, the input data and RDDs are by default persisted as serialized bytes. This reduces both, the memory usage and GC overheads, compared to deserialized persistence. Enabling Kryo serialization further reduces serialized sizes and memory usage. Further reduction in memory usage can be achieved with compression (see the Spark configuration `spark.rdd.compress`), at the cost of CPU time.
+
+* **Clearing old data**: By default, all input data and persisted RDDs generated by DStream transformations are automatically cleared. Spark Streaming decides when to clear the data based on the transformations that are used. For example, if you are using window operation of 10 minutes, then Spark Streaming will keep around last 10 minutes of data, and actively throw away older data.
+Data can be retained for longer duration (e.g. interactively querying older data) by setting `streamingContext.remember`.
+
+* **CMS Garbage Collector**: Use of the concurrent mark-and-sweep GC is strongly recommended for keeping GC-related pauses consistently low. Even though concurrent GC is known to reduce the
overall processing throughput of the system, its use is still recommended to achieve more
-consistent batch processing times.
+consistent batch processing times. Make sure you set the CMS GC on both the driver (using `--driver-java-options` in `spark-submit`) and the executors (using [Spark configuration](configuration.html#runtime-environment) `spark.executor.extraJavaOptions`).
+
+* **Other tips**: To further reduce GC overheads, here are some more tips to try.
+ - Use Tachyon for off-heap storage of persisted RDDs. See more detail in the [Spark Programming Guide](programming-guide.html#rdd-persistence).
+ - Use more executors with smaller heap sizes. This will reduce the GC pressure within each JVM heap.
+
***************************************************************************************************
***************************************************************************************************
# Fault-tolerance Semantics
In this section, we will discuss the behavior of Spark Streaming applications in the event
-of node failures. To understand this, let us remember the basic fault-tolerance semantics of
-Spark's RDDs.
+of failures.
+
+## Background
+{:.no_toc}
+To understand the semantics provided by Spark Streaming, let us remember the basic fault-tolerance semantics of Spark's RDDs.
1. An RDD is an immutable, deterministically re-computable, distributed dataset. Each RDD
remembers the lineage of deterministic operations that were used on a fault-tolerant input
@@ -1854,13 +2131,43 @@ Furthermore, there are two kinds of failures that we should be concerned about:
With this basic knowledge, let us understand the fault-tolerance semantics of Spark Streaming.
-## Semantics with files as input source
+## Definitions
+{:.no_toc}
+The semantics of streaming systems are often captured in terms of how many times each record can be processed by the system. There are three types of guarantees that a system can provide under all possible operating conditions (despite failures, etc.)
+
+1. *At most once*: Each record will be either processed once or not processed at all.
+2. *At least once*: Each record will be processed one or more times. This is stronger than *at-most once* as it ensure that no data will be lost. But there may be duplicates.
+3. *Exactly once*: Each record will be processed exactly once - no data will be lost and no data will be processed multiple times. This is obviously the strongest guarantee of the three.
+
+## Basic Semantics
+{:.no_toc}
+In any stream processing system, broadly speaking, there are three steps in processing the data.
+
+1. *Receiving the data*: The data is received from sources using Receivers or otherwise.
+
+1. *Transforming the data*: The data received data is transformed using DStream and RDD transformations.
+
+1. *Pushing out the data*: The final transformed data is pushed out to external systems like file systems, databases, dashboards, etc.
+
+If a streaming application has to achieve end-to-end exactly-once guarantees, then each step has to provide exactly-once guarantee. That is, each record must be received exactly once, transformed exactly once, and pushed to downstream systems exactly once. Let's understand the semantics of these steps in the context of Spark Streaming.
+
+1. *Receiving the data*: Different input sources provided different guarantees. This is discussed in detail in the next subsection.
+
+1. *Transforming the data*: All data that has been received will be processed _exactly once_, thanks to the guarantees that RDDs provide. Even if there are failures, as long as the received input data is accessible, the final transformed RDDs will always have the same contents.
+
+1. *Pushing out the data*: Output operations by default ensure _at-least once_ semantics because it depends on the type of output operation (idempotent, or not) and the semantics of the downstream system (supports transactions or not). But users can implement their own transaction mechanisms to achieve _exactly-once_ semantics. This is discussed in more details later in the section.
+
+## Semantics of Received Data
+{:.no_toc}
+Different input sources provide different guarantees, ranging from _at-least once_ to _exactly once_. Read for more details.
+
+### With Files
{:.no_toc}
If all of the input data is already present in a fault-tolerant files system like
HDFS, Spark Streaming can always recover from any failure and process all the data. This gives
*exactly-once* semantics, that all the data will be processed exactly once no matter what fails.
-## Semantics with input sources based on receivers
+### With Receiver-based Sources
{:.no_toc}
For input sources based on receivers, the fault-tolerance semantics depend on both the failure
scenario and the type of receiver.
@@ -1879,10 +2186,9 @@ receivers, data received but not replicated can get lost. If the driver node fai
then besides these losses, all the past data that was received and replicated in memory will be
lost. This will affect the results of the stateful transformations.
-To avoid this loss of past received data, Spark 1.2 introduces an experimental feature of _write
+To avoid this loss of past received data, Spark 1.2 introduced _write
ahead logs_ which saves the received data to fault-tolerant storage. With the [write ahead logs
-enabled](#deploying-applications) and reliable receivers, there is zero data loss and
-exactly-once semantics.
+enabled](#deploying-applications) and reliable receivers, there is zero data loss. In terms of semantics, it provides at-least once guarantee.
The following table summarizes the semantics under failures:
@@ -1894,23 +2200,30 @@ The following table summarizes the semantics under failures:
- Spark 1.1 or earlier, or
- Spark 1.2 without write ahead log
+ Spark 1.1 or earlier, OR
+ Spark 1.2 or later without write ahead logs
Buffered data lost with unreliable receivers
- Zero data loss with reliable receivers and files
+ Zero data loss with reliable receivers
+ At-least once semantics
Buffered data lost with unreliable receivers
Past data lost with all receivers
- Zero data loss with files
-
+ Undefined semantics
+
- Spark 1.2 with write ahead log
- Zero data loss with reliable receivers and files
- Zero data loss with reliable receivers and files
+ Spark 1.2 or later with write ahead logs
+
+ Zero data loss with reliable receivers
+ At-least once semantics
+
+
+ Zero data loss with reliable receivers and files
+ At-least once semantics
+
@@ -1919,17 +2232,24 @@ The following table summarizes the semantics under failures:
+### With Kafka Direct API
+{:.no_toc}
+In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that all the Kafka data is received by Spark Streaming exactly once. Along with this, if you implement exactly-once output operation, you can achieve end-to-end exactly-once guarantees. This approach (experimental as of Spark 1.3) is further discussed in the [Kafka Integration Guide](streaming-kafka-integration.html).
+
## Semantics of output operations
{:.no_toc}
-Since all data is modeled as RDDs with their lineage of deterministic operations, any recomputation
- always leads to the same result. As a result, all DStream transformations are guaranteed to have
- _exactly-once_ semantics. That is, the final transformed result will be same even if there were
- was a worker node failure. However, output operations (like `foreachRDD`) have _at-least once_
- semantics, that is, the transformed data may get written to an external entity more than once in
- the event of a worker failure. While this is acceptable for saving to HDFS using the
- `saveAs***Files` operations (as the file will simply get over-written by the same data),
- additional transactions-like mechanisms may be necessary to achieve exactly-once semantics
- for output operations.
+Output operations (like `foreachRDD`) have _at-least once_ semantics, that is,
+the transformed data may get written to an external entity more than once in
+the event of a worker failure. While this is acceptable for saving to file systems using the
+`saveAs***Files` operations (as the file will simply get overwritten with the same data),
+additional effort may be necessary to achieve exactly-once semantics. There are two approaches.
+
+- *Idempotent updates*: Multiple attempts always write the same data. For example, `saveAs***Files` always writes the same data to the generated files.
+
+- *Transactional updates*: All updates are made transactionally so that updates are made exactly once atomically. One way to do this would be the following.
+
+ - Use the batch time (available in `foreachRDD`) and the partition index of the transformed RDD to create an identifier. This identifier uniquely identifies a blob data in the streaming application.
+ - Update external system with this blob transactionally (that is, exactly once, atomically) using the identifier. That is, if the identifier is not already committed, commit the partition data and the identifier atomically. Else if this was already committed, skip the update.
***************************************************************************************************
@@ -1987,7 +2307,11 @@ package and renamed for better clarity.
***************************************************************************************************
# Where to Go from Here
-
+* Additional guides
+ - [Kafka Integration Guide](streaming-kafka-integration.html)
+ - [Flume Integration Guide](streaming-flume-integration.html)
+ - [Kinesis Integration Guide](streaming-kinesis-integration.html)
+ - [Custom Receiver Guide](streaming-custom-receivers.html)
* API documentation
- Scala docs
* [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) and
@@ -2009,8 +2333,8 @@ package and renamed for better clarity.
[ZeroMQUtils](api/java/index.html?org/apache/spark/streaming/zeromq/ZeroMQUtils.html), and
[MQTTUtils](api/java/index.html?org/apache/spark/streaming/mqtt/MQTTUtils.html)
- Python docs
- * [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext)
- * [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream)
+ * [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) and [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream)
+ * [KafkaUtils](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils)
* More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming)
and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/streaming)
diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md
index 14a87f8436984..57b074778f2b0 100644
--- a/docs/submitting-applications.md
+++ b/docs/submitting-applications.md
@@ -174,6 +174,11 @@ This can use up a significant amount of space over time and will need to be clea
is handled automatically, and with Spark standalone, automatic cleanup can be configured with the
`spark.worker.cleanup.appDataTtl` property.
+Users may also include any other dependencies by supplying a comma-delimited list of maven coordinates
+with `--packages`. All transitive dependencies will be handled when using this command. Additional
+repositories (or resolvers in SBT) can be added in a comma-delimited fashion with the flag `--repositories`.
+These commands can be used with `pyspark`, `spark-shell`, and `spark-submit` to include Spark Packages.
+
For Python, the equivalent `--py-files` option can be used to distribute `.egg`, `.zip` and `.py` libraries
to executors.
diff --git a/docs/tuning.md b/docs/tuning.md
index efaac9d3d405f..cbd227868b248 100644
--- a/docs/tuning.md
+++ b/docs/tuning.md
@@ -1,6 +1,8 @@
---
layout: global
-title: Tuning Spark
+displayTitle: Tuning Spark
+title: Tuning
+description: Tuning and performance optimization guide for Spark SPARK_VERSION_SHORT
---
* This will become a table of contents (this text will be scraped).
diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh
index 740c267fd9866..0857657152ec7 100644
--- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh
+++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh
@@ -26,9 +26,10 @@ export SPARK_LOCAL_DIRS="{{spark_local_dirs}}"
export MODULES="{{modules}}"
export SPARK_VERSION="{{spark_version}}"
export SHARK_VERSION="{{shark_version}}"
+export TACHYON_VERSION="{{tachyon_version}}"
export HADOOP_MAJOR_VERSION="{{hadoop_major_version}}"
export SWAP_MB="{{swap}}"
export SPARK_WORKER_INSTANCES="{{spark_worker_instances}}"
export SPARK_MASTER_OPTS="{{spark_master_opts}}"
export AWS_ACCESS_KEY_ID="{{aws_access_key_id}}"
-export AWS_SECRET_ACCESS_KEY="{{aws_secret_access_key}}"
\ No newline at end of file
+export AWS_SECRET_ACCESS_KEY="{{aws_secret_access_key}}"
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index abab209a05ba0..f848874b0c775 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -22,16 +22,20 @@
from __future__ import with_statement
import hashlib
+import itertools
import logging
import os
+import os.path
import pipes
import random
import shutil
import string
+from stat import S_IRUSR
import subprocess
import sys
import tarfile
import tempfile
+import textwrap
import time
import urllib2
import warnings
@@ -39,6 +43,9 @@
from optparse import OptionParser
from sys import stderr
+SPARK_EC2_VERSION = "1.2.1"
+SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__))
+
VALID_SPARK_VERSIONS = set([
"0.7.3",
"0.8.0",
@@ -52,45 +59,81 @@
"1.1.0",
"1.1.1",
"1.2.0",
+ "1.2.1",
])
-DEFAULT_SPARK_VERSION = "1.2.0"
+SPARK_TACHYON_MAP = {
+ "1.0.0": "0.4.1",
+ "1.0.1": "0.4.1",
+ "1.0.2": "0.4.1",
+ "1.1.0": "0.5.0",
+ "1.1.1": "0.5.0",
+ "1.2.0": "0.5.0",
+ "1.2.1": "0.5.0",
+}
+
+DEFAULT_SPARK_VERSION = SPARK_EC2_VERSION
DEFAULT_SPARK_GITHUB_REPO = "https://github.com/apache/spark"
-SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__))
-MESOS_SPARK_EC2_BRANCH = "branch-1.3"
-
-# A URL prefix from which to fetch AMI information
-AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/{b}/ami-list".format(b=MESOS_SPARK_EC2_BRANCH)
-
-
-def setup_boto():
- # Download Boto if it's not already present in the SPARK_EC2_DIR/lib folder:
- version = "boto-2.34.0"
- md5 = "5556223d2d0cc4d06dd4829e671dcecd"
- url = "https://pypi.python.org/packages/source/b/boto/%s.tar.gz" % version
- lib_dir = os.path.join(SPARK_EC2_DIR, "lib")
- if not os.path.exists(lib_dir):
- os.mkdir(lib_dir)
- boto_lib_dir = os.path.join(lib_dir, version)
- if not os.path.isdir(boto_lib_dir):
- tgz_file_path = os.path.join(lib_dir, "%s.tar.gz" % version)
- print "Downloading Boto from PyPi"
- download_stream = urllib2.urlopen(url)
- with open(tgz_file_path, "wb") as tgz_file:
- tgz_file.write(download_stream.read())
- with open(tgz_file_path) as tar:
- if hashlib.md5(tar.read()).hexdigest() != md5:
- print >> stderr, "ERROR: Got wrong md5sum for Boto"
- sys.exit(1)
- tar = tarfile.open(tgz_file_path)
- tar.extractall(path=lib_dir)
- tar.close()
- os.remove(tgz_file_path)
- print "Finished downloading Boto"
- sys.path.insert(0, boto_lib_dir)
+# Default location to get the spark-ec2 scripts (and ami-list) from
+DEFAULT_SPARK_EC2_GITHUB_REPO = "https://github.com/mesos/spark-ec2"
+DEFAULT_SPARK_EC2_BRANCH = "branch-1.3"
+
+
+def setup_external_libs(libs):
+ """
+ Download external libraries from PyPI to SPARK_EC2_DIR/lib/ and prepend them to our PATH.
+ """
+ PYPI_URL_PREFIX = "https://pypi.python.org/packages/source"
+ SPARK_EC2_LIB_DIR = os.path.join(SPARK_EC2_DIR, "lib")
+
+ if not os.path.exists(SPARK_EC2_LIB_DIR):
+ print "Downloading external libraries that spark-ec2 needs from PyPI to {path}...".format(
+ path=SPARK_EC2_LIB_DIR
+ )
+ print "This should be a one-time operation."
+ os.mkdir(SPARK_EC2_LIB_DIR)
+
+ for lib in libs:
+ versioned_lib_name = "{n}-{v}".format(n=lib["name"], v=lib["version"])
+ lib_dir = os.path.join(SPARK_EC2_LIB_DIR, versioned_lib_name)
+
+ if not os.path.isdir(lib_dir):
+ tgz_file_path = os.path.join(SPARK_EC2_LIB_DIR, versioned_lib_name + ".tar.gz")
+ print " - Downloading {lib}...".format(lib=lib["name"])
+ download_stream = urllib2.urlopen(
+ "{prefix}/{first_letter}/{lib_name}/{lib_name}-{lib_version}.tar.gz".format(
+ prefix=PYPI_URL_PREFIX,
+ first_letter=lib["name"][:1],
+ lib_name=lib["name"],
+ lib_version=lib["version"]
+ )
+ )
+ with open(tgz_file_path, "wb") as tgz_file:
+ tgz_file.write(download_stream.read())
+ with open(tgz_file_path) as tar:
+ if hashlib.md5(tar.read()).hexdigest() != lib["md5"]:
+ print >> stderr, "ERROR: Got wrong md5sum for {lib}.".format(lib=lib["name"])
+ sys.exit(1)
+ tar = tarfile.open(tgz_file_path)
+ tar.extractall(path=SPARK_EC2_LIB_DIR)
+ tar.close()
+ os.remove(tgz_file_path)
+ print " - Finished downloading {lib}.".format(lib=lib["name"])
+ sys.path.insert(1, lib_dir)
+
+
+# Only PyPI libraries are supported.
+external_libs = [
+ {
+ "name": "boto",
+ "version": "2.34.0",
+ "md5": "5556223d2d0cc4d06dd4829e671dcecd"
+ }
+]
+
+setup_external_libs(external_libs)
-setup_boto()
import boto
from boto.ec2.blockdevicemapping import BlockDeviceMapping, BlockDeviceType, EBSBlockDeviceType
from boto import ec2
@@ -103,12 +146,11 @@ class UsageError(Exception):
# Configure and parse our command-line arguments
def parse_args():
parser = OptionParser(
- usage="spark-ec2 [options] "
- + "\n\n can be: launch, destroy, login, stop, start, get-master, reboot-slaves",
- add_help_option=False)
- parser.add_option(
- "-h", "--help", action="help",
- help="Show this help message and exit")
+ prog="spark-ec2",
+ version="%prog {v}".format(v=SPARK_EC2_VERSION),
+ usage="%prog [options] \n\n"
+ + " can be: launch, destroy, login, stop, start, get-master, reboot-slaves")
+
parser.add_option(
"-s", "--slaves", type="int", default=1,
help="Number of slaves to launch (default: %default)")
@@ -130,13 +172,15 @@ def parse_args():
help="Master instance type (leave empty for same as instance-type)")
parser.add_option(
"-r", "--region", default="us-east-1",
- help="EC2 region zone to launch instances in")
+ help="EC2 region used to launch instances in, or to find them in (default: %default)")
parser.add_option(
"-z", "--zone", default="",
help="Availability zone to launch instances in, or 'all' to spread " +
"slaves across multiple (an additional $0.01/Gb for bandwidth" +
"between zones applies) (default: a single zone chosen at random)")
- parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use")
+ parser.add_option(
+ "-a", "--ami",
+ help="Amazon Machine Image ID to use")
parser.add_option(
"-v", "--spark-version", default=DEFAULT_SPARK_VERSION,
help="Version of Spark to use: 'X.Y.Z' or a specific git hash (default: %default)")
@@ -144,6 +188,23 @@ def parse_args():
"--spark-git-repo",
default=DEFAULT_SPARK_GITHUB_REPO,
help="Github repo from which to checkout supplied commit hash (default: %default)")
+ parser.add_option(
+ "--spark-ec2-git-repo",
+ default=DEFAULT_SPARK_EC2_GITHUB_REPO,
+ help="Github repo from which to checkout spark-ec2 (default: %default)")
+ parser.add_option(
+ "--spark-ec2-git-branch",
+ default=DEFAULT_SPARK_EC2_BRANCH,
+ help="Github repo branch of spark-ec2 to use (default: %default)")
+ parser.add_option(
+ "--deploy-root-dir",
+ default=None,
+ help="A directory to copy into / on the first master. " +
+ "Must be absolute. Note that a trailing slash is handled as per rsync: " +
+ "If you omit it, the last directory of the --deploy-root-dir path will be created " +
+ "in / before copying its contents. If you append the trailing slash, " +
+ "the directory is not created and its contents are copied directly into /. " +
+ "(default: %default).")
parser.add_option(
"--hadoop-major-version", default="1",
help="Major version of Hadoop (default: %default)")
@@ -168,10 +229,11 @@ def parse_args():
"Only possible on EBS-backed AMIs. " +
"EBS volumes are only attached if --ebs-vol-size > 0." +
"Only support up to 8 EBS volumes.")
- parser.add_option("--placement-group", type="string", default=None,
- help="Which placement group to try and launch " +
- "instances into. Assumes placement group is already " +
- "created.")
+ parser.add_option(
+ "--placement-group", type="string", default=None,
+ help="Which placement group to try and launch " +
+ "instances into. Assumes placement group is already " +
+ "created.")
parser.add_option(
"--swap", metavar="SWAP", type="int", default=1024,
help="Swap space to set up per node, in MB (default: %default)")
@@ -204,7 +266,7 @@ def parse_args():
"(e.g -Dspark.worker.timeout=180)")
parser.add_option(
"--user-data", type="string", default="",
- help="Path to a user-data file (most AMI's interpret this as an initialization script)")
+ help="Path to a user-data file (most AMIs interpret this as an initialization script)")
parser.add_option(
"--authorized-address", type="string", default="0.0.0.0/0",
help="Address to authorize on created security groups (default: %default)")
@@ -215,9 +277,11 @@ def parse_args():
"--copy-aws-credentials", action="store_true", default=False,
help="Add AWS credentials to hadoop configuration to allow Spark to access S3")
parser.add_option(
- "--subnet-id", default=None, help="VPC subnet to launch instances in")
+ "--subnet-id", default=None,
+ help="VPC subnet to launch instances in")
parser.add_option(
- "--vpc-id", default=None, help="VPC to launch instances in")
+ "--vpc-id", default=None,
+ help="VPC to launch instances in")
(opts, args) = parser.parse_args()
if len(args) != 2:
@@ -272,65 +336,69 @@ def get_validate_spark_version(version, repo):
return version
-# Check whether a given EC2 instance object is in a state we consider active,
-# i.e. not terminating or terminated. We count both stopping and stopped as
-# active since we can restart stopped clusters.
-def is_active(instance):
- return (instance.state in ['pending', 'running', 'stopping', 'stopped'])
-
-
-# Attempt to resolve an appropriate AMI given the architecture and region of the request.
# Source: http://aws.amazon.com/amazon-linux-ami/instance-type-matrix/
# Last Updated: 2014-06-20
# For easy maintainability, please keep this manually-inputted dictionary sorted by key.
+EC2_INSTANCE_TYPES = {
+ "c1.medium": "pvm",
+ "c1.xlarge": "pvm",
+ "c3.2xlarge": "pvm",
+ "c3.4xlarge": "pvm",
+ "c3.8xlarge": "pvm",
+ "c3.large": "pvm",
+ "c3.xlarge": "pvm",
+ "cc1.4xlarge": "hvm",
+ "cc2.8xlarge": "hvm",
+ "cg1.4xlarge": "hvm",
+ "cr1.8xlarge": "hvm",
+ "hi1.4xlarge": "pvm",
+ "hs1.8xlarge": "pvm",
+ "i2.2xlarge": "hvm",
+ "i2.4xlarge": "hvm",
+ "i2.8xlarge": "hvm",
+ "i2.xlarge": "hvm",
+ "m1.large": "pvm",
+ "m1.medium": "pvm",
+ "m1.small": "pvm",
+ "m1.xlarge": "pvm",
+ "m2.2xlarge": "pvm",
+ "m2.4xlarge": "pvm",
+ "m2.xlarge": "pvm",
+ "m3.2xlarge": "hvm",
+ "m3.large": "hvm",
+ "m3.medium": "hvm",
+ "m3.xlarge": "hvm",
+ "r3.2xlarge": "hvm",
+ "r3.4xlarge": "hvm",
+ "r3.8xlarge": "hvm",
+ "r3.large": "hvm",
+ "r3.xlarge": "hvm",
+ "t1.micro": "pvm",
+ "t2.medium": "hvm",
+ "t2.micro": "hvm",
+ "t2.small": "hvm",
+}
+
+
+def get_tachyon_version(spark_version):
+ return SPARK_TACHYON_MAP.get(spark_version, "")
+
+
+# Attempt to resolve an appropriate AMI given the architecture and region of the request.
def get_spark_ami(opts):
- instance_types = {
- "c1.medium": "pvm",
- "c1.xlarge": "pvm",
- "c3.2xlarge": "pvm",
- "c3.4xlarge": "pvm",
- "c3.8xlarge": "pvm",
- "c3.large": "pvm",
- "c3.xlarge": "pvm",
- "cc1.4xlarge": "hvm",
- "cc2.8xlarge": "hvm",
- "cg1.4xlarge": "hvm",
- "cr1.8xlarge": "hvm",
- "hi1.4xlarge": "pvm",
- "hs1.8xlarge": "pvm",
- "i2.2xlarge": "hvm",
- "i2.4xlarge": "hvm",
- "i2.8xlarge": "hvm",
- "i2.xlarge": "hvm",
- "m1.large": "pvm",
- "m1.medium": "pvm",
- "m1.small": "pvm",
- "m1.xlarge": "pvm",
- "m2.2xlarge": "pvm",
- "m2.4xlarge": "pvm",
- "m2.xlarge": "pvm",
- "m3.2xlarge": "hvm",
- "m3.large": "hvm",
- "m3.medium": "hvm",
- "m3.xlarge": "hvm",
- "r3.2xlarge": "hvm",
- "r3.4xlarge": "hvm",
- "r3.8xlarge": "hvm",
- "r3.large": "hvm",
- "r3.xlarge": "hvm",
- "t1.micro": "pvm",
- "t2.medium": "hvm",
- "t2.micro": "hvm",
- "t2.small": "hvm",
- }
- if opts.instance_type in instance_types:
- instance_type = instance_types[opts.instance_type]
+ if opts.instance_type in EC2_INSTANCE_TYPES:
+ instance_type = EC2_INSTANCE_TYPES[opts.instance_type]
else:
instance_type = "pvm"
print >> stderr,\
"Don't recognize %s, assuming type is pvm" % opts.instance_type
- ami_path = "%s/%s/%s" % (AMI_PREFIX, opts.region, instance_type)
+ # URL prefix from which to fetch AMI information
+ ami_prefix = "{r}/{b}/ami-list".format(
+ r=opts.spark_ec2_git_repo.replace("https://github.com", "https://raw.github.com", 1),
+ b=opts.spark_ec2_git_branch)
+
+ ami_path = "%s/%s/%s" % (ami_prefix, opts.region, instance_type)
try:
ami = urllib2.urlopen(ami_path).read().strip()
print "Spark AMI: " + ami
@@ -349,6 +417,7 @@ def launch_cluster(conn, opts, cluster_name):
if opts.identity_file is None:
print >> stderr, "ERROR: Must provide an identity file (-i) for ssh connections."
sys.exit(1)
+
if opts.key_pair is None:
print >> stderr, "ERROR: Must provide a key pair name (-k) to use on instances."
sys.exit(1)
@@ -538,8 +607,11 @@ def launch_cluster(conn, opts, cluster_name):
placement_group=opts.placement_group,
user_data=user_data_content)
slave_nodes += slave_res.instances
- print "Launched %d slaves in %s, regid = %s" % (num_slaves_this_zone,
- zone, slave_res.id)
+ print "Launched {s} slave{plural_s} in {z}, regid = {r}".format(
+ s=num_slaves_this_zone,
+ plural_s=('' if num_slaves_this_zone == 1 else 's'),
+ z=zone,
+ r=slave_res.id)
i += 1
# Launch or resume masters
@@ -569,6 +641,9 @@ def launch_cluster(conn, opts, cluster_name):
master_nodes = master_res.instances
print "Launched master in %s, regid = %s" % (zone, master_res.id)
+ # This wait time corresponds to SPARK-4983
+ print "Waiting for AWS to propagate instance metadata..."
+ time.sleep(5)
# Give the instances descriptive names
for master in master_nodes:
master.add_tag(
@@ -583,39 +658,47 @@ def launch_cluster(conn, opts, cluster_name):
return (master_nodes, slave_nodes)
-# Get the EC2 instances in an existing cluster if available.
-# Returns a tuple of lists of EC2 instance objects for the masters and slaves
+def get_existing_cluster(conn, opts, cluster_name, die_on_error=True):
+ """
+ Get the EC2 instances in an existing cluster if available.
+ Returns a tuple of lists of EC2 instance objects for the masters and slaves.
+ """
+ print "Searching for existing cluster {c} in region {r}...".format(
+ c=cluster_name, r=opts.region)
+ def get_instances(group_names):
+ """
+ Get all non-terminated instances that belong to any of the provided security groups.
-def get_existing_cluster(conn, opts, cluster_name, die_on_error=True):
- print "Searching for existing cluster " + cluster_name + "..."
- reservations = conn.get_all_reservations()
- master_nodes = []
- slave_nodes = []
- for res in reservations:
- active = [i for i in res.instances if is_active(i)]
- for inst in active:
- group_names = [g.name for g in inst.groups]
- if (cluster_name + "-master") in group_names:
- master_nodes.append(inst)
- elif (cluster_name + "-slaves") in group_names:
- slave_nodes.append(inst)
- if any((master_nodes, slave_nodes)):
- print "Found %d master(s), %d slaves" % (len(master_nodes), len(slave_nodes))
- if master_nodes != [] or not die_on_error:
- return (master_nodes, slave_nodes)
- else:
- if master_nodes == [] and slave_nodes != []:
- print >> sys.stderr, "ERROR: Could not find master in group " + cluster_name + "-master"
- else:
- print >> sys.stderr, "ERROR: Could not find any existing cluster"
+ EC2 reservation filters and instance states are documented here:
+ http://docs.aws.amazon.com/cli/latest/reference/ec2/describe-instances.html#options
+ """
+ reservations = conn.get_all_reservations(
+ filters={"instance.group-name": group_names})
+ instances = itertools.chain.from_iterable(r.instances for r in reservations)
+ return [i for i in instances if i.state not in ["shutting-down", "terminated"]]
+
+ master_instances = get_instances([cluster_name + "-master"])
+ slave_instances = get_instances([cluster_name + "-slaves"])
+
+ if any((master_instances, slave_instances)):
+ print "Found {m} master{plural_m}, {s} slave{plural_s}.".format(
+ m=len(master_instances),
+ plural_m=('' if len(master_instances) == 1 else 's'),
+ s=len(slave_instances),
+ plural_s=('' if len(slave_instances) == 1 else 's'))
+
+ if not master_instances and die_on_error:
+ print >> sys.stderr, \
+ "ERROR: Could not find a master for cluster {c} in region {r}.".format(
+ c=cluster_name, r=opts.region)
sys.exit(1)
+ return (master_instances, slave_instances)
+
# Deploy configuration files and run setup scripts on a newly launched
# or started EC2 cluster.
-
-
def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key):
master = master_nodes[0].public_dns_name
if deploy_ssh_key:
@@ -643,12 +726,15 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key):
# NOTE: We should clone the repository before running deploy_files to
# prevent ec2-variables.sh from being overwritten
+ print "Cloning spark-ec2 scripts from {r}/tree/{b} on master...".format(
+ r=opts.spark_ec2_git_repo, b=opts.spark_ec2_git_branch)
ssh(
host=master,
opts=opts,
command="rm -rf spark-ec2"
+ " && "
- + "git clone https://github.com/mesos/spark-ec2.git -b {b}".format(b=MESOS_SPARK_EC2_BRANCH)
+ + "git clone {r} -b {b} spark-ec2".format(r=opts.spark_ec2_git_repo,
+ b=opts.spark_ec2_git_branch)
)
print "Deploying files to master..."
@@ -661,6 +747,14 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key):
modules=modules
)
+ if opts.deploy_root_dir is not None:
+ print "Deploying {s} to master...".format(s=opts.deploy_root_dir)
+ deploy_user_files(
+ root_dir=opts.deploy_root_dir,
+ opts=opts,
+ master_nodes=master_nodes
+ )
+
print "Running setup on master..."
setup_spark_cluster(master, opts)
print "Done!"
@@ -675,21 +769,32 @@ def setup_spark_cluster(master, opts):
print "Ganglia started at http://%s:5080/ganglia" % master
-def is_ssh_available(host, opts):
+def is_ssh_available(host, opts, print_ssh_output=True):
"""
Check if SSH is available on a host.
"""
- try:
- with open(os.devnull, 'w') as devnull:
- ret = subprocess.check_call(
- ssh_command(opts) + ['-t', '-t', '-o', 'ConnectTimeout=3',
- '%s@%s' % (opts.user, host), stringify_command('true')],
- stdout=devnull,
- stderr=devnull
- )
- return ret == 0
- except subprocess.CalledProcessError as e:
- return False
+ s = subprocess.Popen(
+ ssh_command(opts) + ['-t', '-t', '-o', 'ConnectTimeout=3',
+ '%s@%s' % (opts.user, host), stringify_command('true')],
+ stdout=subprocess.PIPE,
+ stderr=subprocess.STDOUT # we pipe stderr through stdout to preserve output order
+ )
+ cmd_output = s.communicate()[0] # [1] is stderr, which we redirected to stdout
+
+ if s.returncode != 0 and print_ssh_output:
+ # extra leading newline is for spacing in wait_for_cluster_state()
+ print textwrap.dedent("""\n
+ Warning: SSH connection error. (This could be temporary.)
+ Host: {h}
+ SSH return code: {r}
+ SSH output: {o}
+ """).format(
+ h=host,
+ r=s.returncode,
+ o=cmd_output.strip()
+ )
+
+ return s.returncode == 0
def is_cluster_ssh_available(cluster_instances, opts):
@@ -828,9 +933,13 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules):
if "." in opts.spark_version:
# Pre-built Spark deploy
spark_v = get_validate_spark_version(opts.spark_version, opts.spark_git_repo)
+ tachyon_v = get_tachyon_version(spark_v)
else:
# Spark-only custom deploy
spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version)
+ tachyon_v = ""
+ print "Deploying Spark via git hash; Tachyon won't be set up"
+ modules = filter(lambda x: x != "tachyon", modules)
template_vars = {
"master_list": '\n'.join([i.public_dns_name for i in master_nodes]),
@@ -843,6 +952,7 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules):
"swap": str(opts.swap),
"modules": '\n'.join(modules),
"spark_version": spark_v,
+ "tachyon_version": tachyon_v,
"hadoop_major_version": opts.hadoop_major_version,
"spark_worker_instances": "%d" % opts.worker_instances,
"spark_master_opts": opts.master_opts
@@ -887,6 +997,23 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules):
shutil.rmtree(tmp_dir)
+# Deploy a given local directory to a cluster, WITHOUT parameter substitution.
+# Note that unlike deploy_files, this works for binary files.
+# Also, it is up to the user to add (or not) the trailing slash in root_dir.
+# Files are only deployed to the first master instance in the cluster.
+#
+# root_dir should be an absolute path.
+def deploy_user_files(root_dir, opts, master_nodes):
+ active_master = master_nodes[0].public_dns_name
+ command = [
+ 'rsync', '-rv',
+ '-e', stringify_command(ssh_command(opts)),
+ "%s" % root_dir,
+ "%s@%s:/" % (opts.user, active_master)
+ ]
+ subprocess.check_call(command)
+
+
def stringify_command(parts):
if isinstance(parts, str):
return parts
@@ -896,6 +1023,7 @@ def stringify_command(parts):
def ssh_args(opts):
parts = ['-o', 'StrictHostKeyChecking=no']
+ parts += ['-o', 'UserKnownHostsFile=/dev/null']
if opts.identity_file is not None:
parts += ['-i', opts.identity_file]
return parts
@@ -1003,10 +1131,65 @@ def real_main():
DeprecationWarning
)
+ if opts.identity_file is not None:
+ if not os.path.exists(opts.identity_file):
+ print >> stderr,\
+ "ERROR: The identity file '{f}' doesn't exist.".format(f=opts.identity_file)
+ sys.exit(1)
+
+ file_mode = os.stat(opts.identity_file).st_mode
+ if not (file_mode & S_IRUSR) or not oct(file_mode)[-2:] == '00':
+ print >> stderr, "ERROR: The identity file must be accessible only by you."
+ print >> stderr, 'You can fix this with: chmod 400 "{f}"'.format(f=opts.identity_file)
+ sys.exit(1)
+
+ if opts.instance_type not in EC2_INSTANCE_TYPES:
+ print >> stderr, "Warning: Unrecognized EC2 instance type for instance-type: {t}".format(
+ t=opts.instance_type)
+
+ if opts.master_instance_type != "":
+ if opts.master_instance_type not in EC2_INSTANCE_TYPES:
+ print >> stderr, \
+ "Warning: Unrecognized EC2 instance type for master-instance-type: {t}".format(
+ t=opts.master_instance_type)
+ # Since we try instance types even if we can't resolve them, we check if they resolve first
+ # and, if they do, see if they resolve to the same virtualization type.
+ if opts.instance_type in EC2_INSTANCE_TYPES and \
+ opts.master_instance_type in EC2_INSTANCE_TYPES:
+ if EC2_INSTANCE_TYPES[opts.instance_type] != \
+ EC2_INSTANCE_TYPES[opts.master_instance_type]:
+ print >> stderr, \
+ "Error: spark-ec2 currently does not support having a master and slaves with " + \
+ "different AMI virtualization types."
+ print >> stderr, "master instance virtualization type: {t}".format(
+ t=EC2_INSTANCE_TYPES[opts.master_instance_type])
+ print >> stderr, "slave instance virtualization type: {t}".format(
+ t=EC2_INSTANCE_TYPES[opts.instance_type])
+ sys.exit(1)
+
if opts.ebs_vol_num > 8:
print >> stderr, "ebs-vol-num cannot be greater than 8"
sys.exit(1)
+ # Prevent breaking ami_prefix (/, .git and startswith checks)
+ # Prevent forks with non spark-ec2 names for now.
+ if opts.spark_ec2_git_repo.endswith("/") or \
+ opts.spark_ec2_git_repo.endswith(".git") or \
+ not opts.spark_ec2_git_repo.startswith("https://github.com") or \
+ not opts.spark_ec2_git_repo.endswith("spark-ec2"):
+ print >> stderr, "spark-ec2-git-repo must be a github repo and it must not have a " \
+ "trailing / or .git. " \
+ "Furthermore, we currently only support forks named spark-ec2."
+ sys.exit(1)
+
+ if not (opts.deploy_root_dir is None or
+ (os.path.isabs(opts.deploy_root_dir) and
+ os.path.isdir(opts.deploy_root_dir) and
+ os.path.exists(opts.deploy_root_dir))):
+ print >> stderr, "--deploy-root-dir must be an absolute path to a directory that exists " \
+ "on the local file system"
+ sys.exit(1)
+
try:
conn = ec2.connect_to_region(opts.region)
except Exception as e:
@@ -1034,14 +1217,16 @@ def real_main():
setup_cluster(conn, master_nodes, slave_nodes, opts, True)
elif action == "destroy":
- 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
+ if any(master_nodes + slave_nodes):
+ print "The following instances will be terminated:"
+ for inst in master_nodes + slave_nodes:
+ print "> %s" % inst.public_dns_name
+ print "ALL DATA ON ALL NODES WILL BE LOST!!"
+
+ msg = "Are you sure you want to destroy the cluster {c}? (y/N) ".format(c=cluster_name)
response = raw_input(msg)
if response == "y":
print "Terminating master..."
@@ -1053,7 +1238,6 @@ def real_main():
# Delete security groups as well
if opts.delete_groups:
- print "Deleting security groups (this will take some time)..."
group_names = [cluster_name + "-master", cluster_name + "-slaves"]
wait_for_cluster_state(
conn=conn,
@@ -1061,6 +1245,7 @@ def real_main():
cluster_instances=(master_nodes + slave_nodes),
cluster_state='terminated'
)
+ print "Deleting security groups (this will take some time)..."
attempt = 1
while attempt <= 3:
print "Attempt %d" % attempt
@@ -1082,11 +1267,12 @@ def real_main():
time.sleep(30) # Yes, it does have to be this long :-(
for group in groups:
try:
- conn.delete_security_group(group.name)
- print "Deleted security group " + group.name
+ # It is needed to use group_id to make it work with VPC
+ conn.delete_security_group(group_id=group.id)
+ print "Deleted security group %s" % group.name
except boto.exception.EC2ResponseError:
success = False
- print "Failed to delete security group " + group.name
+ print "Failed to delete security group %s" % group.name
# Unfortunately, group.revoke() returns True even if a rule was not
# deleted, so this needs to be rerun if something fails
@@ -1166,6 +1352,17 @@ def real_main():
cluster_instances=(master_nodes + slave_nodes),
cluster_state='ssh-ready'
)
+
+ # Determine types of running instances
+ existing_master_type = master_nodes[0].instance_type
+ existing_slave_type = slave_nodes[0].instance_type
+ # Setting opts.master_instance_type to the empty string indicates we
+ # have the same instance type for the master and the slaves
+ if existing_master_type == existing_slave_type:
+ existing_master_type = ""
+ opts.master_instance_type = existing_master_type
+ opts.instance_type = existing_slave_type
+
setup_cluster(conn, master_nodes, slave_nodes, opts, False)
else:
diff --git a/examples/pom.xml b/examples/pom.xml
index 8caad2bc2e27a..994071d94d0ad 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../pom.xml
diff --git a/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java b/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java
new file mode 100644
index 0000000000000..bab9f2478e779
--- /dev/null
+++ b/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.streaming;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Arrays;
+import java.util.regex.Pattern;
+
+import scala.Tuple2;
+
+import com.google.common.collect.Lists;
+import kafka.serializer.StringDecoder;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.function.*;
+import org.apache.spark.streaming.api.java.*;
+import org.apache.spark.streaming.kafka.KafkaUtils;
+import org.apache.spark.streaming.Durations;
+
+/**
+ * Consumes messages from one or more topics in Kafka and does wordcount.
+ * Usage: DirectKafkaWordCount
+ * is a list of one or more Kafka brokers
+ * is a list of one or more kafka topics to consume from
+ *
+ * Example:
+ * $ bin/run-example streaming.KafkaWordCount broker1-host:port,broker2-host:port topic1,topic2
+ */
+
+public final class JavaDirectKafkaWordCount {
+ private static final Pattern SPACE = Pattern.compile(" ");
+
+ public static void main(String[] args) {
+ if (args.length < 2) {
+ System.err.println("Usage: DirectKafkaWordCount \n" +
+ " is a list of one or more Kafka brokers\n" +
+ " is a list of one or more kafka topics to consume from\n\n");
+ System.exit(1);
+ }
+
+ StreamingExamples.setStreamingLogLevels();
+
+ String brokers = args[0];
+ String topics = args[1];
+
+ // Create context with 2 second batch interval
+ SparkConf sparkConf = new SparkConf().setAppName("JavaDirectKafkaWordCount");
+ JavaStreamingContext jssc = new JavaStreamingContext(sparkConf, Durations.seconds(2));
+
+ HashSet topicsSet = new HashSet(Arrays.asList(topics.split(",")));
+ HashMap kafkaParams = new HashMap();
+ kafkaParams.put("metadata.broker.list", brokers);
+
+ // Create direct kafka stream with brokers and topics
+ JavaPairInputDStream messages = KafkaUtils.createDirectStream(
+ jssc,
+ String.class,
+ String.class,
+ StringDecoder.class,
+ StringDecoder.class,
+ kafkaParams,
+ topicsSet
+ );
+
+ // Get the lines, split them into words, count the words and print
+ JavaDStream lines = messages.map(new Function, String>() {
+ @Override
+ public String call(Tuple2 tuple2) {
+ return tuple2._2();
+ }
+ });
+ JavaDStream words = lines.flatMap(new FlatMapFunction() {
+ @Override
+ public Iterable call(String x) {
+ return Lists.newArrayList(SPACE.split(x));
+ }
+ });
+ JavaPairDStream wordCounts = words.mapToPair(
+ new PairFunction() {
+ @Override
+ public Tuple2 call(String s) {
+ return new Tuple2(s, 1);
+ }
+ }).reduceByKey(
+ new Function2() {
+ @Override
+ public Integer call(Integer i1, Integer i2) {
+ return i1 + i2;
+ }
+ });
+ wordCounts.print();
+
+ // Start the computation
+ jssc.start();
+ jssc.awaitTermination();
+ }
+}
diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala
new file mode 100644
index 0000000000000..1c8a20bf8f1ae
--- /dev/null
+++ b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.streaming
+
+import kafka.serializer.StringDecoder
+
+import org.apache.spark.streaming._
+import org.apache.spark.streaming.kafka._
+import org.apache.spark.SparkConf
+
+/**
+ * Consumes messages from one or more topics in Kafka and does wordcount.
+ * Usage: DirectKafkaWordCount
+ * is a list of one or more Kafka brokers
+ * is a list of one or more kafka topics to consume from
+ *
+ * Example:
+ * $ bin/run-example streaming.DirectKafkaWordCount broker1-host:port,broker2-host:port \
+ * topic1,topic2
+ */
+object DirectKafkaWordCount {
+ def main(args: Array[String]) {
+ if (args.length < 2) {
+ System.err.println(s"""
+ |Usage: DirectKafkaWordCount
+ | is a list of one or more Kafka brokers
+ | is a list of one or more kafka topics to consume from
+ |
+ """".stripMargin)
+ System.exit(1)
+ }
+
+ StreamingExamples.setStreamingLogLevels()
+
+ val Array(brokers, topics) = args
+
+ // Create context with 2 second batch interval
+ val sparkConf = new SparkConf().setAppName("DirectKafkaWordCount")
+ val ssc = new StreamingContext(sparkConf, Seconds(2))
+
+ // Create direct kafka stream with brokers and topics
+ val topicsSet = topics.split(",").toSet
+ val kafkaParams = Map[String, String]("metadata.broker.list" -> brokers)
+ val messages = KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder](
+ ssc, kafkaParams, topicsSet)
+
+ // Get the lines, split them into words, count the words and print
+ val lines = messages.map(_._2)
+ val words = lines.flatMap(_.split(" "))
+ val wordCounts = words.map(x => (x, 1L)).reduceByKey(_ + _)
+ wordCounts.print()
+
+ // Start the computation
+ ssc.start()
+ ssc.awaitTermination()
+ }
+}
diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java
index 0fbee6e433608..9bbc14ea40875 100644
--- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java
+++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java
@@ -34,8 +34,8 @@
import org.apache.spark.ml.tuning.CrossValidatorModel;
import org.apache.spark.ml.tuning.ParamGridBuilder;
import org.apache.spark.sql.DataFrame;
-import org.apache.spark.sql.SQLContext;
import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
/**
* A simple example demonstrating model selection using CrossValidator.
@@ -71,7 +71,7 @@ public static void main(String[] args) {
new LabeledDocument(9L, "a e c l", 0.0),
new LabeledDocument(10L, "spark compile", 1.0),
new LabeledDocument(11L, "hadoop software", 0.0));
- DataFrame training = jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class);
+ DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class);
// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr.
Tokenizer tokenizer = new Tokenizer()
@@ -112,14 +112,15 @@ public static void main(String[] args) {
new Document(5L, "l m n"),
new Document(6L, "mapreduce spark"),
new Document(7L, "apache hadoop"));
- DataFrame test = jsql.applySchema(jsc.parallelize(localTest), Document.class);
+ DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class);
// Make predictions on test documents. cvModel uses the best model found (lrModel).
- cvModel.transform(test).registerTempTable("prediction");
- DataFrame predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction");
- for (Row r: predictions.collect()) {
- System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2)
+ DataFrame predictions = cvModel.transform(test);
+ for (Row r: predictions.select("id", "text", "probability", "prediction").collect()) {
+ System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2)
+ ", prediction=" + r.get(3));
}
+
+ jsc.stop();
}
}
diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java
new file mode 100644
index 0000000000000..19d0eb216848e
--- /dev/null
+++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.ml;
+
+import java.util.List;
+
+import com.google.common.collect.Lists;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.ml.classification.Classifier;
+import org.apache.spark.ml.classification.ClassificationModel;
+import org.apache.spark.ml.param.IntParam;
+import org.apache.spark.ml.param.ParamMap;
+import org.apache.spark.ml.param.Params;
+import org.apache.spark.ml.param.Params$;
+import org.apache.spark.mllib.linalg.BLAS;
+import org.apache.spark.mllib.linalg.Vector;
+import org.apache.spark.mllib.linalg.Vectors;
+import org.apache.spark.mllib.regression.LabeledPoint;
+import org.apache.spark.sql.DataFrame;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
+
+
+/**
+ * A simple example demonstrating how to write your own learning algorithm using Estimator,
+ * Transformer, and other abstractions.
+ * This mimics {@link org.apache.spark.ml.classification.LogisticRegression}.
+ *
+ * Run with
+ *
+ * bin/run-example ml.JavaDeveloperApiExample
+ *
+ */
+public class JavaDeveloperApiExample {
+
+ public static void main(String[] args) throws Exception {
+ SparkConf conf = new SparkConf().setAppName("JavaDeveloperApiExample");
+ JavaSparkContext jsc = new JavaSparkContext(conf);
+ SQLContext jsql = new SQLContext(jsc);
+
+ // Prepare training data.
+ List localTraining = Lists.newArrayList(
+ new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)),
+ new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)),
+ new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)),
+ new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)));
+ DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledPoint.class);
+
+ // Create a LogisticRegression instance. This instance is an Estimator.
+ MyJavaLogisticRegression lr = new MyJavaLogisticRegression();
+ // Print out the parameters, documentation, and any default values.
+ System.out.println("MyJavaLogisticRegression parameters:\n" + lr.explainParams() + "\n");
+
+ // We may set parameters using setter methods.
+ lr.setMaxIter(10);
+
+ // Learn a LogisticRegression model. This uses the parameters stored in lr.
+ MyJavaLogisticRegressionModel model = lr.fit(training);
+
+ // Prepare test data.
+ List localTest = Lists.newArrayList(
+ new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)),
+ new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)),
+ new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)));
+ DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class);
+
+ // Make predictions on test documents. cvModel uses the best model found (lrModel).
+ DataFrame results = model.transform(test);
+ double sumPredictions = 0;
+ for (Row r : results.select("features", "label", "prediction").collect()) {
+ sumPredictions += r.getDouble(2);
+ }
+ if (sumPredictions != 0.0) {
+ throw new Exception("MyJavaLogisticRegression predicted something other than 0," +
+ " even though all weights are 0!");
+ }
+
+ jsc.stop();
+ }
+}
+
+/**
+ * Example of defining a type of {@link Classifier}.
+ *
+ * NOTE: This is private since it is an example. In practice, you may not want it to be private.
+ */
+class MyJavaLogisticRegression
+ extends Classifier
+ implements Params {
+
+ /**
+ * Param for max number of iterations
+ *
+ * NOTE: The usual way to add a parameter to a model or algorithm is to include:
+ * - val myParamName: ParamType
+ * - def getMyParamName
+ * - def setMyParamName
+ */
+ IntParam maxIter = new IntParam(this, "maxIter", "max number of iterations");
+
+ int getMaxIter() { return (Integer) get(maxIter); }
+
+ public MyJavaLogisticRegression() {
+ setMaxIter(100);
+ }
+
+ // The parameter setter is in this class since it should return type MyJavaLogisticRegression.
+ MyJavaLogisticRegression setMaxIter(int value) {
+ return (MyJavaLogisticRegression) set(maxIter, value);
+ }
+
+ // This method is used by fit().
+ // In Java, we have to make it public since Java does not understand Scala's protected modifier.
+ public MyJavaLogisticRegressionModel train(DataFrame dataset, ParamMap paramMap) {
+ // Extract columns from data using helper method.
+ JavaRDD oldDataset = extractLabeledPoints(dataset, paramMap).toJavaRDD();
+
+ // Do learning to estimate the weight vector.
+ int numFeatures = oldDataset.take(1).get(0).features().size();
+ Vector weights = Vectors.zeros(numFeatures); // Learning would happen here.
+
+ // Create a model, and return it.
+ return new MyJavaLogisticRegressionModel(this, paramMap, weights);
+ }
+}
+
+/**
+ * Example of defining a type of {@link ClassificationModel}.
+ *
+ * NOTE: This is private since it is an example. In practice, you may not want it to be private.
+ */
+class MyJavaLogisticRegressionModel
+ extends ClassificationModel implements Params {
+
+ private MyJavaLogisticRegression parent_;
+ public MyJavaLogisticRegression parent() { return parent_; }
+
+ private ParamMap fittingParamMap_;
+ public ParamMap fittingParamMap() { return fittingParamMap_; }
+
+ private Vector weights_;
+ public Vector weights() { return weights_; }
+
+ public MyJavaLogisticRegressionModel(
+ MyJavaLogisticRegression parent_,
+ ParamMap fittingParamMap_,
+ Vector weights_) {
+ this.parent_ = parent_;
+ this.fittingParamMap_ = fittingParamMap_;
+ this.weights_ = weights_;
+ }
+
+ // This uses the default implementation of transform(), which reads column "features" and outputs
+ // columns "prediction" and "rawPrediction."
+
+ // This uses the default implementation of predict(), which chooses the label corresponding to
+ // the maximum value returned by [[predictRaw()]].
+
+ /**
+ * Raw prediction for each possible label.
+ * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives
+ * a measure of confidence in each possible label (where larger = more confident).
+ * This internal method is used to implement [[transform()]] and output [[rawPredictionCol]].
+ *
+ * @return vector where element i is the raw prediction for label i.
+ * This raw prediction may be any real number, where a larger value indicates greater
+ * confidence for that label.
+ *
+ * In Java, we have to make this method public since Java does not understand Scala's protected
+ * modifier.
+ */
+ public Vector predictRaw(Vector features) {
+ double margin = BLAS.dot(features, weights_);
+ // There are 2 classes (binary classification), so we return a length-2 vector,
+ // where index i corresponds to class i (i = 0, 1).
+ return Vectors.dense(-margin, margin);
+ }
+
+ /**
+ * Number of classes the label can take. 2 indicates binary classification.
+ */
+ public int numClasses() { return 2; }
+
+ /**
+ * Create a copy of the model.
+ * The copy is shallow, except for the embedded paramMap, which gets a deep copy.
+ *
+ * This is used for the defaul implementation of [[transform()]].
+ *
+ * In Java, we have to make this method public since Java does not understand Scala's protected
+ * modifier.
+ */
+ public MyJavaLogisticRegressionModel copy() {
+ MyJavaLogisticRegressionModel m =
+ new MyJavaLogisticRegressionModel(parent_, fittingParamMap_, weights_);
+ Params$.MODULE$.inheritValues(this.paramMap(), this, m);
+ return m;
+ }
+}
diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java
index eaaa344be49c8..4e02acce696e6 100644
--- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java
+++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java
@@ -29,8 +29,8 @@
import org.apache.spark.mllib.linalg.Vectors;
import org.apache.spark.mllib.regression.LabeledPoint;
import org.apache.spark.sql.DataFrame;
-import org.apache.spark.sql.SQLContext;
import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
/**
* A simple example demonstrating ways to specify parameters for Estimators and Transformers.
@@ -54,7 +54,7 @@ public static void main(String[] args) {
new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)),
new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)),
new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)));
- DataFrame training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class);
+ DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledPoint.class);
// Create a LogisticRegression instance. This instance is an Estimator.
LogisticRegression lr = new LogisticRegression();
@@ -81,7 +81,7 @@ public static void main(String[] args) {
// One can also combine ParamMaps.
ParamMap paramMap2 = new ParamMap();
- paramMap2.put(lr.scoreCol().w("probability")); // Change output column name
+ paramMap2.put(lr.probabilityCol().w("myProbability")); // Change output column name
ParamMap paramMapCombined = paramMap.$plus$plus(paramMap2);
// Now learn a new model using the paramMapCombined parameters.
@@ -94,18 +94,18 @@ public static void main(String[] args) {
new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)),
new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)),
new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)));
- DataFrame test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class);
+ DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class);
// Make predictions on test documents using the Transformer.transform() method.
// LogisticRegression.transform will only use the 'features' column.
- // Note that model2.transform() outputs a 'probability' column instead of the usual 'score'
- // column since we renamed the lr.scoreCol parameter previously.
- model2.transform(test).registerTempTable("results");
- DataFrame results =
- jsql.sql("SELECT features, label, probability, prediction FROM results");
- for (Row r: results.collect()) {
+ // Note that model2.transform() outputs a 'myProbability' column instead of the usual
+ // 'probability' column since we renamed the lr.probabilityCol parameter previously.
+ DataFrame results = model2.transform(test);
+ for (Row r: results.select("features", "label", "myProbability", "prediction").collect()) {
System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2)
+ ", prediction=" + r.get(3));
}
+
+ jsc.stop();
}
}
diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java
index 82d665a3e1386..ef1ec103a879f 100644
--- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java
+++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java
@@ -30,8 +30,8 @@
import org.apache.spark.ml.feature.HashingTF;
import org.apache.spark.ml.feature.Tokenizer;
import org.apache.spark.sql.DataFrame;
-import org.apache.spark.sql.SQLContext;
import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
/**
* A simple text classification pipeline that recognizes "spark" from input text. It uses the Java
@@ -54,7 +54,7 @@ public static void main(String[] args) {
new LabeledDocument(1L, "b d", 0.0),
new LabeledDocument(2L, "spark f g h", 1.0),
new LabeledDocument(3L, "hadoop mapreduce", 0.0));
- DataFrame training = jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class);
+ DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class);
// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr.
Tokenizer tokenizer = new Tokenizer()
@@ -79,14 +79,15 @@ public static void main(String[] args) {
new Document(5L, "l m n"),
new Document(6L, "mapreduce spark"),
new Document(7L, "apache hadoop"));
- DataFrame test = jsql.applySchema(jsc.parallelize(localTest), Document.class);
+ DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class);
// Make predictions on test documents.
- model.transform(test).registerTempTable("prediction");
- DataFrame predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction");
- for (Row r: predictions.collect()) {
- System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2)
+ DataFrame predictions = model.transform(test);
+ for (Row r: predictions.select("id", "text", "probability", "prediction").collect()) {
+ System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2)
+ ", prediction=" + r.get(3));
}
+
+ jsc.stop();
}
}
diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java
new file mode 100644
index 0000000000000..36baf5868736c
--- /dev/null
+++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.mllib;
+
+import java.util.ArrayList;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.mllib.fpm.FPGrowth;
+import org.apache.spark.mllib.fpm.FPGrowthModel;
+
+/**
+ * Java example for mining frequent itemsets using FP-growth.
+ * Example usage: ./bin/run-example mllib.JavaFPGrowthExample ./data/mllib/sample_fpgrowth.txt
+ */
+public class JavaFPGrowthExample {
+
+ public static void main(String[] args) {
+ String inputFile;
+ double minSupport = 0.3;
+ int numPartition = -1;
+ if (args.length < 1) {
+ System.err.println(
+ "Usage: JavaFPGrowth [minSupport] [numPartition]");
+ System.exit(1);
+ }
+ inputFile = args[0];
+ if (args.length >= 2) {
+ minSupport = Double.parseDouble(args[1]);
+ }
+ if (args.length >= 3) {
+ numPartition = Integer.parseInt(args[2]);
+ }
+
+ SparkConf sparkConf = new SparkConf().setAppName("JavaFPGrowthExample");
+ JavaSparkContext sc = new JavaSparkContext(sparkConf);
+
+ JavaRDD> transactions = sc.textFile(inputFile).map(
+ new Function>() {
+ @Override
+ public ArrayList call(String s) {
+ return Lists.newArrayList(s.split(" "));
+ }
+ }
+ );
+
+ FPGrowthModel model = new FPGrowth()
+ .setMinSupport(minSupport)
+ .setNumPartitions(numPartition)
+ .run(transactions);
+
+ for (FPGrowth.FreqItemset s: model.freqItemsets().toJavaRDD().collect()) {
+ System.out.println("[" + Joiner.on(",").join(s.javaItems()) + "], " + s.freq());
+ }
+
+ sc.stop();
+ }
+}
diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java
new file mode 100644
index 0000000000000..36207ae38d9a9
--- /dev/null
+++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.mllib;
+
+import scala.Tuple2;
+
+import org.apache.spark.api.java.*;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.mllib.clustering.DistributedLDAModel;
+import org.apache.spark.mllib.clustering.LDA;
+import org.apache.spark.mllib.linalg.Matrix;
+import org.apache.spark.mllib.linalg.Vector;
+import org.apache.spark.mllib.linalg.Vectors;
+import org.apache.spark.SparkConf;
+
+public class JavaLDAExample {
+ public static void main(String[] args) {
+ SparkConf conf = new SparkConf().setAppName("LDA Example");
+ JavaSparkContext sc = new JavaSparkContext(conf);
+
+ // Load and parse the data
+ String path = "data/mllib/sample_lda_data.txt";
+ JavaRDD data = sc.textFile(path);
+ JavaRDD parsedData = data.map(
+ new Function() {
+ public Vector call(String s) {
+ String[] sarray = s.trim().split(" ");
+ double[] values = new double[sarray.length];
+ for (int i = 0; i < sarray.length; i++)
+ values[i] = Double.parseDouble(sarray[i]);
+ return Vectors.dense(values);
+ }
+ }
+ );
+ // Index documents with unique IDs
+ JavaPairRDD corpus = JavaPairRDD.fromJavaRDD(parsedData.zipWithIndex().map(
+ new Function, Tuple2>() {
+ public Tuple2 call(Tuple2 doc_id) {
+ return doc_id.swap();
+ }
+ }
+ ));
+ corpus.cache();
+
+ // Cluster the documents into three topics using LDA
+ DistributedLDAModel ldaModel = new LDA().setK(3).run(corpus);
+
+ // Output topics. Each is a distribution over words (matching word count vectors)
+ System.out.println("Learned topics (as distributions over vocab of " + ldaModel.vocabSize()
+ + " words):");
+ Matrix topics = ldaModel.topicsMatrix();
+ for (int topic = 0; topic < 3; topic++) {
+ System.out.print("Topic " + topic + ":");
+ for (int word = 0; word < ldaModel.vocabSize(); word++) {
+ System.out.print(" " + topics.apply(word, topic));
+ }
+ System.out.println();
+ }
+ sc.stop();
+ }
+}
diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java
new file mode 100644
index 0000000000000..6c6f9768f015e
--- /dev/null
+++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.mllib;
+
+import scala.Tuple3;
+
+import com.google.common.collect.Lists;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.mllib.clustering.PowerIterationClustering;
+import org.apache.spark.mllib.clustering.PowerIterationClusteringModel;
+
+/**
+ * Java example for graph clustering using power iteration clustering (PIC).
+ */
+public class JavaPowerIterationClusteringExample {
+ public static void main(String[] args) {
+ SparkConf sparkConf = new SparkConf().setAppName("JavaPowerIterationClusteringExample");
+ JavaSparkContext sc = new JavaSparkContext(sparkConf);
+
+ @SuppressWarnings("unchecked")
+ JavaRDD> similarities = sc.parallelize(Lists.newArrayList(
+ new Tuple3(0L, 1L, 0.9),
+ new Tuple3(1L, 2L, 0.9),
+ new Tuple3(2L, 3L, 0.9),
+ new Tuple3(3L, 4L, 0.1),
+ new Tuple3(4L, 5L, 0.9)));
+
+ PowerIterationClustering pic = new PowerIterationClustering()
+ .setK(2)
+ .setMaxIterations(10);
+ PowerIterationClusteringModel model = pic.run(similarities);
+
+ for (PowerIterationClustering.Assignment a: model.assignments().toJavaRDD().collect()) {
+ System.out.println(a.id() + " -> " + a.cluster());
+ }
+
+ sc.stop();
+ }
+}
diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java
index 8defb769ffaaf..dee794840a3e1 100644
--- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java
+++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java
@@ -74,7 +74,7 @@ public Person call(String line) {
});
// Apply a schema to an RDD of Java Beans and register it as a table.
- DataFrame schemaPeople = sqlCtx.applySchema(people, Person.class);
+ DataFrame schemaPeople = sqlCtx.createDataFrame(people, Person.class);
schemaPeople.registerTempTable("people");
// SQL can be run over RDDs that have been registered as tables.
diff --git a/core/src/test/scala/org/apache/spark/util/FakeClock.scala b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecord.java
similarity index 71%
rename from core/src/test/scala/org/apache/spark/util/FakeClock.scala
rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaRecord.java
index 0a45917b08dd2..e63697a79f23a 100644
--- a/core/src/test/scala/org/apache/spark/util/FakeClock.scala
+++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecord.java
@@ -15,12 +15,17 @@
* limitations under the License.
*/
-package org.apache.spark.util
+package org.apache.spark.examples.streaming;
-class FakeClock extends Clock {
- private var time = 0L
+/** Java Bean class to be used with the example JavaSqlNetworkWordCount. */
+public class JavaRecord implements java.io.Serializable {
+ private String word;
- def advance(millis: Long): Unit = time += millis
+ public String getWord() {
+ return word;
+ }
- def getTime(): Long = time
+ public void setWord(String word) {
+ this.word = word;
+ }
}
diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java
new file mode 100644
index 0000000000000..46562ddbbcb57
--- /dev/null
+++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.streaming;
+
+import java.util.regex.Pattern;
+
+import com.google.common.collect.Lists;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.SparkContext;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.DataFrame;
+import org.apache.spark.api.java.StorageLevels;
+import org.apache.spark.streaming.Durations;
+import org.apache.spark.streaming.Time;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaReceiverInputDStream;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+
+/**
+ * Use DataFrames and SQL to count words in UTF8 encoded, '\n' delimited text received from the
+ * network every second.
+ *
+ * Usage: JavaSqlNetworkWordCount
+ * and describe the TCP server that Spark Streaming would connect to receive data.
+ *
+ * To run this on your local machine, you need to first run a Netcat server
+ * `$ nc -lk 9999`
+ * and then run the example
+ * `$ bin/run-example org.apache.spark.examples.streaming.JavaSqlNetworkWordCount localhost 9999`
+ */
+
+public final class JavaSqlNetworkWordCount {
+ private static final Pattern SPACE = Pattern.compile(" ");
+
+ public static void main(String[] args) {
+ if (args.length < 2) {
+ System.err.println("Usage: JavaNetworkWordCount ");
+ System.exit(1);
+ }
+
+ StreamingExamples.setStreamingLogLevels();
+
+ // Create the context with a 1 second batch size
+ SparkConf sparkConf = new SparkConf().setAppName("JavaSqlNetworkWordCount");
+ JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, Durations.seconds(1));
+
+ // Create a JavaReceiverInputDStream on target ip:port and count the
+ // words in input stream of \n delimited text (eg. generated by 'nc')
+ // Note that no duplication in storage level only for running locally.
+ // Replication necessary in distributed scenario for fault tolerance.
+ JavaReceiverInputDStream lines = ssc.socketTextStream(
+ args[0], Integer.parseInt(args[1]), StorageLevels.MEMORY_AND_DISK_SER);
+ JavaDStream words = lines.flatMap(new FlatMapFunction() {
+ @Override
+ public Iterable call(String x) {
+ return Lists.newArrayList(SPACE.split(x));
+ }
+ });
+
+ // Convert RDDs of the words DStream to DataFrame and run SQL query
+ words.foreachRDD(new Function2, Time, Void>() {
+ @Override
+ public Void call(JavaRDD rdd, Time time) {
+ SQLContext sqlContext = JavaSQLContextSingleton.getInstance(rdd.context());
+
+ // Convert JavaRDD[String] to JavaRDD[bean class] to DataFrame
+ JavaRDD rowRDD = rdd.map(new Function() {
+ public JavaRecord call(String word) {
+ JavaRecord record = new JavaRecord();
+ record.setWord(word);
+ return record;
+ }
+ });
+ DataFrame wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRecord.class);
+
+ // Register as table
+ wordsDataFrame.registerTempTable("words");
+
+ // Do word count on table using SQL and print it
+ DataFrame wordCountsDataFrame =
+ sqlContext.sql("select word, count(*) as total from words group by word");
+ System.out.println("========= " + time + "=========");
+ wordCountsDataFrame.show();
+ return null;
+ }
+ });
+
+ ssc.start();
+ ssc.awaitTermination();
+ }
+}
+
+/** Lazily instantiated singleton instance of SQLContext */
+class JavaSQLContextSingleton {
+ static private transient SQLContext instance = null;
+ static public SQLContext getInstance(SparkContext sparkContext) {
+ if (instance == null) {
+ instance = new SQLContext(sparkContext);
+ }
+ return instance;
+ }
+}
diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java
new file mode 100644
index 0000000000000..dbf2ef02d7b76
--- /dev/null
+++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.streaming;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.regex.Pattern;
+
+import scala.Tuple2;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+
+import org.apache.spark.HashPartitioner;
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.StorageLevels;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.streaming.Durations;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaPairDStream;
+import org.apache.spark.streaming.api.java.JavaReceiverInputDStream;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+
+/**
+ * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every
+ * second starting with initial value of word count.
+ * Usage: JavaStatefulNetworkWordCount
+ * and describe the TCP server that Spark Streaming would connect to receive
+ * data.
+ *
+ * To run this on your local machine, you need to first run a Netcat server
+ * `$ nc -lk 9999`
+ * and then run the example
+ * `$ bin/run-example
+ * org.apache.spark.examples.streaming.JavaStatefulNetworkWordCount localhost 9999`
+ */
+public class JavaStatefulNetworkWordCount {
+ private static final Pattern SPACE = Pattern.compile(" ");
+
+ public static void main(String[] args) {
+ if (args.length < 2) {
+ System.err.println("Usage: JavaStatefulNetworkWordCount ");
+ System.exit(1);
+ }
+
+ StreamingExamples.setStreamingLogLevels();
+
+ // Update the cumulative count function
+ final Function2, Optional, Optional> updateFunction =
+ new Function2, Optional, Optional>() {
+ @Override
+ public Optional call(List values, Optional state) {
+ Integer newSum = state.or(0);
+ for (Integer value : values) {
+ newSum += value;
+ }
+ return Optional.of(newSum);
+ }
+ };
+
+ // Create the context with a 1 second batch size
+ SparkConf sparkConf = new SparkConf().setAppName("JavaStatefulNetworkWordCount");
+ JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, Durations.seconds(1));
+ ssc.checkpoint(".");
+
+ // Initial RDD input to updateStateByKey
+ @SuppressWarnings("unchecked")
+ List> tuples = Arrays.asList(new Tuple2("hello", 1),
+ new Tuple2("world", 1));
+ JavaPairRDD initialRDD = ssc.sc().parallelizePairs(tuples);
+
+ JavaReceiverInputDStream lines = ssc.socketTextStream(
+ args[0], Integer.parseInt(args[1]), StorageLevels.MEMORY_AND_DISK_SER_2);
+
+ JavaDStream words = lines.flatMap(new FlatMapFunction() {
+ @Override
+ public Iterable call(String x) {
+ return Lists.newArrayList(SPACE.split(x));
+ }
+ });
+
+ JavaPairDStream wordsDstream = words.mapToPair(
+ new PairFunction() {
+ @Override
+ public Tuple2 call(String s) {
+ return new Tuple2(s, 1);
+ }
+ });
+
+ // This will give a Dstream made of state (which is the cumulative count of the words)
+ JavaPairDStream stateDstream = wordsDstream.updateStateByKey(updateFunction,
+ new HashPartitioner(ssc.sc().defaultParallelism()), initialRDD);
+
+ stateDstream.print();
+ ssc.start();
+ ssc.awaitTermination();
+ }
+}
diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py
index c7df3d7b74767..d281f4fa44282 100644
--- a/examples/src/main/python/ml/simple_text_classification_pipeline.py
+++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py
@@ -16,10 +16,10 @@
#
from pyspark import SparkContext
-from pyspark.sql import SQLContext, Row
from pyspark.ml import Pipeline
-from pyspark.ml.feature import HashingTF, Tokenizer
from pyspark.ml.classification import LogisticRegression
+from pyspark.ml.feature import HashingTF, Tokenizer
+from pyspark.sql import Row, SQLContext
"""
@@ -36,43 +36,33 @@
sqlCtx = SQLContext(sc)
# Prepare training documents, which are labeled.
- LabeledDocument = Row('id', 'text', 'label')
- training = sqlCtx.inferSchema(
- sc.parallelize([(0L, "a b c d e spark", 1.0),
- (1L, "b d", 0.0),
- (2L, "spark f g h", 1.0),
- (3L, "hadoop mapreduce", 0.0)])
- .map(lambda x: LabeledDocument(*x)))
+ LabeledDocument = Row("id", "text", "label")
+ training = sc.parallelize([(0L, "a b c d e spark", 1.0),
+ (1L, "b d", 0.0),
+ (2L, "spark f g h", 1.0),
+ (3L, "hadoop mapreduce", 0.0)]) \
+ .map(lambda x: LabeledDocument(*x)).toDF()
# Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr.
- tokenizer = Tokenizer() \
- .setInputCol("text") \
- .setOutputCol("words")
- hashingTF = HashingTF() \
- .setInputCol(tokenizer.getOutputCol()) \
- .setOutputCol("features")
- lr = LogisticRegression() \
- .setMaxIter(10) \
- .setRegParam(0.01)
- pipeline = Pipeline() \
- .setStages([tokenizer, hashingTF, lr])
+ tokenizer = Tokenizer(inputCol="text", outputCol="words")
+ hashingTF = HashingTF(inputCol=tokenizer.getOutputCol(), outputCol="features")
+ lr = LogisticRegression(maxIter=10, regParam=0.01)
+ pipeline = Pipeline(stages=[tokenizer, hashingTF, lr])
# Fit the pipeline to training documents.
model = pipeline.fit(training)
# Prepare test documents, which are unlabeled.
- Document = Row('id', 'text')
- test = sqlCtx.inferSchema(
- sc.parallelize([(4L, "spark i j k"),
- (5L, "l m n"),
- (6L, "mapreduce spark"),
- (7L, "apache hadoop")])
- .map(lambda x: Document(*x)))
+ Document = Row("id", "text")
+ test = sc.parallelize([(4L, "spark i j k"),
+ (5L, "l m n"),
+ (6L, "mapreduce spark"),
+ (7L, "apache hadoop")]) \
+ .map(lambda x: Document(*x)).toDF()
# Make predictions on test documents and print columns of interest.
prediction = model.transform(test)
- prediction.registerTempTable("prediction")
- selected = sqlCtx.sql("SELECT id, text, prediction from prediction")
+ selected = prediction.select("id", "text", "prediction")
for row in selected.collect():
print row
diff --git a/examples/src/main/python/mllib/gaussian_mixture_model.py b/examples/src/main/python/mllib/gaussian_mixture_model.py
new file mode 100644
index 0000000000000..a2cd626c9f19d
--- /dev/null
+++ b/examples/src/main/python/mllib/gaussian_mixture_model.py
@@ -0,0 +1,65 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""
+A Gaussian Mixture Model clustering program using MLlib.
+"""
+import sys
+import random
+import argparse
+import numpy as np
+
+from pyspark import SparkConf, SparkContext
+from pyspark.mllib.clustering import GaussianMixture
+
+
+def parseVector(line):
+ return np.array([float(x) for x in line.split(' ')])
+
+
+if __name__ == "__main__":
+ """
+ Parameters
+ ----------
+ :param inputFile: Input file path which contains data points
+ :param k: Number of mixture components
+ :param convergenceTol: Convergence threshold. Default to 1e-3
+ :param maxIterations: Number of EM iterations to perform. Default to 100
+ :param seed: Random seed
+ """
+
+ parser = argparse.ArgumentParser()
+ parser.add_argument('inputFile', help='Input File')
+ parser.add_argument('k', type=int, help='Number of clusters')
+ parser.add_argument('--convergenceTol', default=1e-3, type=float, help='convergence threshold')
+ parser.add_argument('--maxIterations', default=100, type=int, help='Number of iterations')
+ parser.add_argument('--seed', default=random.getrandbits(19),
+ type=long, help='Random seed')
+ args = parser.parse_args()
+
+ conf = SparkConf().setAppName("GMM")
+ sc = SparkContext(conf=conf)
+
+ lines = sc.textFile(args.inputFile)
+ data = lines.map(parseVector)
+ model = GaussianMixture.train(data, args.k, args.convergenceTol,
+ args.maxIterations, args.seed)
+ for i in range(args.k):
+ print ("weight = ", model.weights[i], "mu = ", model.gaussians[i].mu,
+ "sigma = ", model.gaussians[i].sigma.toArray())
+ print ("Cluster labels (first 100): ", model.predict(data).take(100))
+ sc.stop()
diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py
index 7f5c68e3d0fe2..47202fde7510b 100644
--- a/examples/src/main/python/sql.py
+++ b/examples/src/main/python/sql.py
@@ -31,7 +31,7 @@
Row(name="Smith", age=23),
Row(name="Sarah", age=18)])
# Infer schema from the first row, create a DataFrame and print the schema
- some_df = sqlContext.inferSchema(some_rdd)
+ some_df = sqlContext.createDataFrame(some_rdd)
some_df.printSchema()
# Another RDD is created from a list of tuples
@@ -40,7 +40,7 @@
schema = StructType([StructField("person_name", StringType(), False),
StructField("person_age", IntegerType(), False)])
# Create a DataFrame by applying the schema to the RDD and print the schema
- another_df = sqlContext.applySchema(another_rdd, schema)
+ another_df = sqlContext.createDataFrame(another_rdd, schema)
another_df.printSchema()
# root
# |-- age: integer (nullable = true)
diff --git a/examples/src/main/python/status_api_demo.py b/examples/src/main/python/status_api_demo.py
new file mode 100644
index 0000000000000..a33bdc475a06d
--- /dev/null
+++ b/examples/src/main/python/status_api_demo.py
@@ -0,0 +1,67 @@
+#
+# 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.
+#
+
+import time
+import threading
+import Queue
+
+from pyspark import SparkConf, SparkContext
+
+
+def delayed(seconds):
+ def f(x):
+ time.sleep(seconds)
+ return x
+ return f
+
+
+def call_in_background(f, *args):
+ result = Queue.Queue(1)
+ t = threading.Thread(target=lambda: result.put(f(*args)))
+ t.daemon = True
+ t.start()
+ return result
+
+
+def main():
+ conf = SparkConf().set("spark.ui.showConsoleProgress", "false")
+ sc = SparkContext(appName="PythonStatusAPIDemo", conf=conf)
+
+ def run():
+ rdd = sc.parallelize(range(10), 10).map(delayed(2))
+ reduced = rdd.map(lambda x: (x, 1)).reduceByKey(lambda x, y: x + y)
+ return reduced.map(delayed(2)).collect()
+
+ result = call_in_background(run)
+ status = sc.statusTracker()
+ while result.empty():
+ ids = status.getJobIdsForGroup()
+ for id in ids:
+ job = status.getJobInfo(id)
+ print "Job", id, "status: ", job.status
+ for sid in job.stageIds:
+ info = status.getStageInfo(sid)
+ if info:
+ print "Stage %d: %d tasks total (%d active, %d complete)" % \
+ (sid, info.numTasks, info.numActiveTasks, info.numCompletedTasks)
+ time.sleep(1)
+
+ print "Job results are:", result.get()
+ sc.stop()
+
+if __name__ == "__main__":
+ main()
diff --git a/examples/src/main/python/streaming/kafka_wordcount.py b/examples/src/main/python/streaming/kafka_wordcount.py
new file mode 100644
index 0000000000000..51e1ff822fc55
--- /dev/null
+++ b/examples/src/main/python/streaming/kafka_wordcount.py
@@ -0,0 +1,54 @@
+#
+# 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.
+#
+
+"""
+ Counts words in UTF8 encoded, '\n' delimited text received from the network every second.
+ Usage: kafka_wordcount.py
+
+ To run this on your local machine, you need to setup Kafka and create a producer first, see
+ http://kafka.apache.org/documentation.html#quickstart
+
+ and then run the example
+ `$ bin/spark-submit --jars external/kafka-assembly/target/scala-*/\
+ spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py \
+ localhost:2181 test`
+"""
+
+import sys
+
+from pyspark import SparkContext
+from pyspark.streaming import StreamingContext
+from pyspark.streaming.kafka import KafkaUtils
+
+if __name__ == "__main__":
+ if len(sys.argv) != 3:
+ print >> sys.stderr, "Usage: kafka_wordcount.py "
+ exit(-1)
+
+ sc = SparkContext(appName="PythonStreamingKafkaWordCount")
+ ssc = StreamingContext(sc, 1)
+
+ zkQuorum, topic = sys.argv[1:]
+ kvs = KafkaUtils.createStream(ssc, zkQuorum, "spark-streaming-consumer", {topic: 1})
+ lines = kvs.map(lambda x: x[1])
+ counts = lines.flatMap(lambda line: line.split(" ")) \
+ .map(lambda word: (word, 1)) \
+ .reduceByKey(lambda a, b: a+b)
+ counts.pprint()
+
+ ssc.start()
+ ssc.awaitTermination()
diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py
new file mode 100644
index 0000000000000..f89bc562d856b
--- /dev/null
+++ b/examples/src/main/python/streaming/sql_network_wordcount.py
@@ -0,0 +1,82 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""
+ Use DataFrames and SQL to count words in UTF8 encoded, '\n' delimited text received from the
+ network every second.
+
+ Usage: sql_network_wordcount.py
+ and describe the TCP server that Spark Streaming would connect to receive data.
+
+ To run this on your local machine, you need to first run a Netcat server
+ `$ nc -lk 9999`
+ and then run the example
+ `$ bin/spark-submit examples/src/main/python/streaming/sql_network_wordcount.py localhost 9999`
+"""
+
+import os
+import sys
+
+from pyspark import SparkContext
+from pyspark.streaming import StreamingContext
+from pyspark.sql import SQLContext, Row
+
+
+def getSqlContextInstance(sparkContext):
+ if ('sqlContextSingletonInstance' not in globals()):
+ globals()['sqlContextSingletonInstance'] = SQLContext(sparkContext)
+ return globals()['sqlContextSingletonInstance']
+
+
+if __name__ == "__main__":
+ if len(sys.argv) != 3:
+ print >> sys.stderr, "Usage: sql_network_wordcount.py "
+ exit(-1)
+ host, port = sys.argv[1:]
+ sc = SparkContext(appName="PythonSqlNetworkWordCount")
+ ssc = StreamingContext(sc, 1)
+
+ # Create a socket stream on target ip:port and count the
+ # words in input stream of \n delimited text (eg. generated by 'nc')
+ lines = ssc.socketTextStream(host, int(port))
+ words = lines.flatMap(lambda line: line.split(" "))
+
+ # Convert RDDs of the words DStream to DataFrame and run SQL query
+ def process(time, rdd):
+ print "========= %s =========" % str(time)
+
+ try:
+ # Get the singleton instance of SQLContext
+ sqlContext = getSqlContextInstance(rdd.context)
+
+ # Convert RDD[String] to RDD[Row] to DataFrame
+ rowRdd = rdd.map(lambda w: Row(word=w))
+ wordsDataFrame = sqlContext.createDataFrame(rowRdd)
+
+ # Register as table
+ wordsDataFrame.registerTempTable("words")
+
+ # Do word count on table using SQL and print it
+ wordCountsDataFrame = \
+ sqlContext.sql("select word, count(*) as total from words group by word")
+ wordCountsDataFrame.show()
+ except:
+ pass
+
+ words.foreachRDD(process)
+ ssc.start()
+ ssc.awaitTermination()
diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
index 1b53f3edbe92e..4c129dbe2d12d 100644
--- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
@@ -29,7 +29,7 @@ object BroadcastTest {
val blockSize = if (args.length > 3) args(3) else "4096"
val sparkConf = new SparkConf().setAppName("Broadcast Test")
- .set("spark.broadcast.factory", s"org.apache.spark.broadcast.${bcName}BroaddcastFactory")
+ .set("spark.broadcast.factory", s"org.apache.spark.broadcast.${bcName}BroadcastFactory")
.set("spark.broadcast.blockSize", blockSize)
val sc = new SparkContext(sparkConf)
diff --git a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
index 65251e93190f0..e757283823fc3 100644
--- a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
@@ -19,6 +19,8 @@ package org.apache.spark.examples
import scala.collection.JavaConversions._
+import org.apache.spark.util.Utils
+
/** Prints out environmental information, sleeps, and then exits. Made to
* test driver submission in the standalone scheduler. */
object DriverSubmissionTest {
@@ -30,7 +32,7 @@ object DriverSubmissionTest {
val numSecondsToSleep = args(0).toInt
val env = System.getenv()
- val properties = System.getProperties()
+ val properties = Utils.getSystemProperties
println("Environment variables containing SPARK_TEST:")
env.filter{case (k, v) => k.contains("SPARK_TEST")}.foreach(println)
diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
index 822673347bdce..f4684b42b5d41 100644
--- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
@@ -18,7 +18,7 @@
package org.apache.spark.examples
import org.apache.hadoop.hbase.client.HBaseAdmin
-import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor}
+import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor, TableName}
import org.apache.hadoop.hbase.mapreduce.TableInputFormat
import org.apache.spark._
@@ -36,7 +36,7 @@ object HBaseTest {
// Initialize hBase table if necessary
val admin = new HBaseAdmin(conf)
if (!admin.isTableAvailable(args(0))) {
- val tableDesc = new HTableDescriptor(args(0))
+ val tableDesc = new HTableDescriptor(TableName.valueOf(args(0)))
admin.createTable(tableDesc)
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala
index e809a65b79975..f6f8d9f90c275 100644
--- a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala
@@ -17,11 +17,6 @@
package org.apache.spark.examples.graphx
-import org.apache.spark.SparkContext._
-import org.apache.spark._
-import org.apache.spark.graphx._
-
-
/**
* Uses GraphX to run PageRank on a LiveJournal social network graph. Download the dataset from
* http://snap.stanford.edu/data/soc-LiveJournal1.html.
@@ -31,13 +26,13 @@ object LiveJournalPageRank {
if (args.length < 1) {
System.err.println(
"Usage: LiveJournalPageRank \n" +
+ " --numEPart=\n" +
+ " The number of partitions for the graph's edge RDD.\n" +
" [--tol=]\n" +
" The tolerance allowed at convergence (smaller => more accurate). Default is " +
"0.001.\n" +
" [--output=]\n" +
" If specified, the file to write the ranks to.\n" +
- " [--numEPart=]\n" +
- " The number of partitions for the graph's edge RDD. Default is 4.\n" +
" [--partStrategy=RandomVertexCut | EdgePartition1D | EdgePartition2D | " +
"CanonicalRandomVertexCut]\n" +
" The way edges are assigned to edge partitions. Default is RandomVertexCut.")
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala
index 283bb80f1c788..6c0af20461d3b 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala
@@ -23,6 +23,7 @@ import org.apache.spark.ml.classification.LogisticRegression
import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator
import org.apache.spark.ml.feature.{HashingTF, Tokenizer}
import org.apache.spark.ml.tuning.{ParamGridBuilder, CrossValidator}
+import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.sql.{Row, SQLContext}
/**
@@ -43,10 +44,10 @@ object CrossValidatorExample {
val conf = new SparkConf().setAppName("CrossValidatorExample")
val sc = new SparkContext(conf)
val sqlContext = new SQLContext(sc)
- import sqlContext._
+ import sqlContext.implicits._
// Prepare training documents, which are labeled.
- val training = sparkContext.parallelize(Seq(
+ val training = sc.parallelize(Seq(
LabeledDocument(0L, "a b c d e spark", 1.0),
LabeledDocument(1L, "b d", 0.0),
LabeledDocument(2L, "spark f g h", 1.0),
@@ -89,21 +90,21 @@ object CrossValidatorExample {
crossval.setNumFolds(2) // Use 3+ in practice
// Run cross-validation, and choose the best set of parameters.
- val cvModel = crossval.fit(training)
+ val cvModel = crossval.fit(training.toDF())
// Prepare test documents, which are unlabeled.
- val test = sparkContext.parallelize(Seq(
+ val test = sc.parallelize(Seq(
Document(4L, "spark i j k"),
Document(5L, "l m n"),
Document(6L, "mapreduce spark"),
Document(7L, "apache hadoop")))
// Make predictions on test documents. cvModel uses the best model found (lrModel).
- cvModel.transform(test)
- .select("id", "text", "score", "prediction")
+ cvModel.transform(test.toDF())
+ .select("id", "text", "probability", "prediction")
.collect()
- .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) =>
- println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction)
+ .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) =>
+ println(s"($id, $text) --> prob=$prob, prediction=$prediction")
}
sc.stop()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala
new file mode 100644
index 0000000000000..df26798e41b7b
--- /dev/null
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.ml
+
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.ml.classification.{Classifier, ClassifierParams, ClassificationModel}
+import org.apache.spark.ml.param.{Params, IntParam, ParamMap}
+import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors}
+import org.apache.spark.mllib.regression.LabeledPoint
+import org.apache.spark.sql.{DataFrame, Row, SQLContext}
+
+
+/**
+ * A simple example demonstrating how to write your own learning algorithm using Estimator,
+ * Transformer, and other abstractions.
+ * This mimics [[org.apache.spark.ml.classification.LogisticRegression]].
+ * Run with
+ * {{{
+ * bin/run-example ml.DeveloperApiExample
+ * }}}
+ */
+object DeveloperApiExample {
+
+ def main(args: Array[String]) {
+ val conf = new SparkConf().setAppName("DeveloperApiExample")
+ val sc = new SparkContext(conf)
+ val sqlContext = new SQLContext(sc)
+ import sqlContext.implicits._
+
+ // Prepare training data.
+ val training = sc.parallelize(Seq(
+ LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)),
+ LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)),
+ LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)),
+ LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))))
+
+ // Create a LogisticRegression instance. This instance is an Estimator.
+ val lr = new MyLogisticRegression()
+ // Print out the parameters, documentation, and any default values.
+ println("MyLogisticRegression parameters:\n" + lr.explainParams() + "\n")
+
+ // We may set parameters using setter methods.
+ lr.setMaxIter(10)
+
+ // Learn a LogisticRegression model. This uses the parameters stored in lr.
+ val model = lr.fit(training.toDF())
+
+ // Prepare test data.
+ val test = sc.parallelize(Seq(
+ LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)),
+ LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)),
+ LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))))
+
+ // Make predictions on test data.
+ val sumPredictions: Double = model.transform(test.toDF())
+ .select("features", "label", "prediction")
+ .collect()
+ .map { case Row(features: Vector, label: Double, prediction: Double) =>
+ prediction
+ }.sum
+ assert(sumPredictions == 0.0,
+ "MyLogisticRegression predicted something other than 0, even though all weights are 0!")
+
+ sc.stop()
+ }
+}
+
+/**
+ * Example of defining a parameter trait for a user-defined type of [[Classifier]].
+ *
+ * NOTE: This is private since it is an example. In practice, you may not want it to be private.
+ */
+private trait MyLogisticRegressionParams extends ClassifierParams {
+
+ /**
+ * Param for max number of iterations
+ *
+ * NOTE: The usual way to add a parameter to a model or algorithm is to include:
+ * - val myParamName: ParamType
+ * - def getMyParamName
+ * - def setMyParamName
+ * Here, we have a trait to be mixed in with the Estimator and Model (MyLogisticRegression
+ * and MyLogisticRegressionModel). We place the setter (setMaxIter) method in the Estimator
+ * class since the maxIter parameter is only used during training (not in the Model).
+ */
+ val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations")
+ def getMaxIter: Int = get(maxIter)
+}
+
+/**
+ * Example of defining a type of [[Classifier]].
+ *
+ * NOTE: This is private since it is an example. In practice, you may not want it to be private.
+ */
+private class MyLogisticRegression
+ extends Classifier[Vector, MyLogisticRegression, MyLogisticRegressionModel]
+ with MyLogisticRegressionParams {
+
+ setMaxIter(100) // Initialize
+
+ // The parameter setter is in this class since it should return type MyLogisticRegression.
+ def setMaxIter(value: Int): this.type = set(maxIter, value)
+
+ // This method is used by fit()
+ override protected def train(
+ dataset: DataFrame,
+ paramMap: ParamMap): MyLogisticRegressionModel = {
+ // Extract columns from data using helper method.
+ val oldDataset = extractLabeledPoints(dataset, paramMap)
+
+ // Do learning to estimate the weight vector.
+ val numFeatures = oldDataset.take(1)(0).features.size
+ val weights = Vectors.zeros(numFeatures) // Learning would happen here.
+
+ // Create a model, and return it.
+ new MyLogisticRegressionModel(this, paramMap, weights)
+ }
+}
+
+/**
+ * Example of defining a type of [[ClassificationModel]].
+ *
+ * NOTE: This is private since it is an example. In practice, you may not want it to be private.
+ */
+private class MyLogisticRegressionModel(
+ override val parent: MyLogisticRegression,
+ override val fittingParamMap: ParamMap,
+ val weights: Vector)
+ extends ClassificationModel[Vector, MyLogisticRegressionModel]
+ with MyLogisticRegressionParams {
+
+ // This uses the default implementation of transform(), which reads column "features" and outputs
+ // columns "prediction" and "rawPrediction."
+
+ // This uses the default implementation of predict(), which chooses the label corresponding to
+ // the maximum value returned by [[predictRaw()]].
+
+ /**
+ * Raw prediction for each possible label.
+ * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives
+ * a measure of confidence in each possible label (where larger = more confident).
+ * This internal method is used to implement [[transform()]] and output [[rawPredictionCol]].
+ *
+ * @return vector where element i is the raw prediction for label i.
+ * This raw prediction may be any real number, where a larger value indicates greater
+ * confidence for that label.
+ */
+ override protected def predictRaw(features: Vector): Vector = {
+ val margin = BLAS.dot(features, weights)
+ // There are 2 classes (binary classification), so we return a length-2 vector,
+ // where index i corresponds to class i (i = 0, 1).
+ Vectors.dense(-margin, margin)
+ }
+
+ /** Number of classes the label can take. 2 indicates binary classification. */
+ override val numClasses: Int = 2
+
+ /**
+ * Create a copy of the model.
+ * The copy is shallow, except for the embedded paramMap, which gets a deep copy.
+ *
+ * This is used for the defaul implementation of [[transform()]].
+ */
+ override protected def copy(): MyLogisticRegressionModel = {
+ val m = new MyLogisticRegressionModel(parent, fittingParamMap, weights)
+ Params.inheritValues(this.paramMap, this, m)
+ m
+ }
+}
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala
index b7885829459a3..25f21113bf622 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala
@@ -93,8 +93,8 @@ object MovieLensALS {
| bin/spark-submit --class org.apache.spark.examples.ml.MovieLensALS \
| examples/target/scala-*/spark-examples-*.jar \
| --rank 10 --maxIter 15 --regParam 0.1 \
- | --movies path/to/movielens/movies.dat \
- | --ratings path/to/movielens/ratings.dat
+ | --movies data/mllib/als/sample_movielens_movies.txt \
+ | --ratings data/mllib/als/sample_movielens_ratings.txt
""".stripMargin)
}
@@ -109,7 +109,7 @@ object MovieLensALS {
val conf = new SparkConf().setAppName(s"MovieLensALS with $params")
val sc = new SparkContext(conf)
val sqlContext = new SQLContext(sc)
- import sqlContext._
+ import sqlContext.implicits._
val ratings = sc.textFile(params.ratings).map(Rating.parseRating).cache()
@@ -137,9 +137,9 @@ object MovieLensALS {
.setRegParam(params.regParam)
.setNumBlocks(params.numBlocks)
- val model = als.fit(training)
+ val model = als.fit(training.toDF())
- val predictions = model.transform(test).cache()
+ val predictions = model.transform(test.toDF()).cache()
// Evaluate the model.
// TODO: Create an evaluator to compute RMSE.
@@ -157,17 +157,23 @@ object MovieLensALS {
println(s"Test RMSE = $rmse.")
// Inspect false positives.
- predictions.registerTempTable("prediction")
- sc.textFile(params.movies).map(Movie.parseMovie).registerTempTable("movie")
- sqlContext.sql(
- """
- |SELECT userId, prediction.movieId, title, rating, prediction
- | FROM prediction JOIN movie ON prediction.movieId = movie.movieId
- | WHERE rating <= 1 AND prediction >= 4
- | LIMIT 100
- """.stripMargin)
- .collect()
- .foreach(println)
+ // Note: We reference columns in 2 ways:
+ // (1) predictions("movieId") lets us specify the movieId column in the predictions
+ // DataFrame, rather than the movieId column in the movies DataFrame.
+ // (2) $"userId" specifies the userId column in the predictions DataFrame.
+ // We could also write predictions("userId") but do not have to since
+ // the movies DataFrame does not have a column "userId."
+ val movies = sc.textFile(params.movies).map(Movie.parseMovie).toDF()
+ val falsePositives = predictions.join(movies)
+ .where((predictions("movieId") === movies("movieId"))
+ && ($"rating" <= 1) && ($"prediction" >= 4))
+ .select($"userId", predictions("movieId"), $"title", $"rating", $"prediction")
+ val numFalsePositives = falsePositives.count()
+ println(s"Found $numFalsePositives false positives")
+ if (numFalsePositives > 0) {
+ println(s"Example false positives:")
+ falsePositives.limit(100).collect().foreach(println)
+ }
sc.stop()
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala
index 95cc9801eaeb9..bf805149d0af6 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala
@@ -37,12 +37,12 @@ object SimpleParamsExample {
val conf = new SparkConf().setAppName("SimpleParamsExample")
val sc = new SparkContext(conf)
val sqlContext = new SQLContext(sc)
- import sqlContext._
+ import sqlContext.implicits._
// Prepare training data.
- // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of Java Beans
- // into DataFrames, where it uses the bean metadata to infer the schema.
- val training = sparkContext.parallelize(Seq(
+ // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes
+ // into DataFrames, where it uses the case class metadata to infer the schema.
+ val training = sc.parallelize(Seq(
LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)),
LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)),
LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)),
@@ -58,7 +58,7 @@ object SimpleParamsExample {
.setRegParam(0.01)
// Learn a LogisticRegression model. This uses the parameters stored in lr.
- val model1 = lr.fit(training)
+ val model1 = lr.fit(training.toDF())
// Since model1 is a Model (i.e., a Transformer produced by an Estimator),
// we can view the parameters it used during fit().
// This prints the parameter (name: value) pairs, where names are unique IDs for this
@@ -72,29 +72,29 @@ object SimpleParamsExample {
paramMap.put(lr.regParam -> 0.1, lr.threshold -> 0.55) // Specify multiple Params.
// One can also combine ParamMaps.
- val paramMap2 = ParamMap(lr.scoreCol -> "probability") // Change output column name
+ val paramMap2 = ParamMap(lr.probabilityCol -> "myProbability") // Change output column name
val paramMapCombined = paramMap ++ paramMap2
// Now learn a new model using the paramMapCombined parameters.
// paramMapCombined overrides all parameters set earlier via lr.set* methods.
- val model2 = lr.fit(training, paramMapCombined)
+ val model2 = lr.fit(training.toDF(), paramMapCombined)
println("Model 2 was fit using parameters: " + model2.fittingParamMap)
- // Prepare test documents.
- val test = sparkContext.parallelize(Seq(
+ // Prepare test data.
+ val test = sc.parallelize(Seq(
LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)),
LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)),
LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))))
- // Make predictions on test documents using the Transformer.transform() method.
+ // Make predictions on test data using the Transformer.transform() method.
// LogisticRegression.transform will only use the 'features' column.
- // Note that model2.transform() outputs a 'probability' column instead of the usual 'score'
- // column since we renamed the lr.scoreCol parameter previously.
- model2.transform(test)
- .select("features", "label", "probability", "prediction")
+ // Note that model2.transform() outputs a 'myProbability' column instead of the usual
+ // 'probability' column since we renamed the lr.probabilityCol parameter previously.
+ model2.transform(test.toDF())
+ .select("features", "label", "myProbability", "prediction")
.collect()
- .foreach { case Row(features: Vector, label: Double, prob: Double, prediction: Double) =>
- println("(" + features + ", " + label + ") -> prob=" + prob + ", prediction=" + prediction)
+ .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) =>
+ println(s"($features, $label) -> prob=$prob, prediction=$prediction")
}
sc.stop()
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala
index 065db62b0f5ed..6772efd2c581c 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala
@@ -23,6 +23,7 @@ import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.ml.Pipeline
import org.apache.spark.ml.classification.LogisticRegression
import org.apache.spark.ml.feature.{HashingTF, Tokenizer}
+import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.sql.{Row, SQLContext}
@BeanInfo
@@ -44,10 +45,10 @@ object SimpleTextClassificationPipeline {
val conf = new SparkConf().setAppName("SimpleTextClassificationPipeline")
val sc = new SparkContext(conf)
val sqlContext = new SQLContext(sc)
- import sqlContext._
+ import sqlContext.implicits._
// Prepare training documents, which are labeled.
- val training = sparkContext.parallelize(Seq(
+ val training = sc.parallelize(Seq(
LabeledDocument(0L, "a b c d e spark", 1.0),
LabeledDocument(1L, "b d", 0.0),
LabeledDocument(2L, "spark f g h", 1.0),
@@ -68,21 +69,21 @@ object SimpleTextClassificationPipeline {
.setStages(Array(tokenizer, hashingTF, lr))
// Fit the pipeline to training documents.
- val model = pipeline.fit(training)
+ val model = pipeline.fit(training.toDF())
// Prepare test documents, which are unlabeled.
- val test = sparkContext.parallelize(Seq(
+ val test = sc.parallelize(Seq(
Document(4L, "spark i j k"),
Document(5L, "l m n"),
Document(6L, "mapreduce spark"),
Document(7L, "apache hadoop")))
// Make predictions on test documents.
- model.transform(test)
- .select("id", "text", "score", "prediction")
+ model.transform(test.toDF())
+ .select("id", "text", "probability", "prediction")
.collect()
- .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) =>
- println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction)
+ .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) =>
+ println(s"($id, $text) --> prob=$prob, prediction=$prediction")
}
sc.stop()
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala
index ab58375649d25..e943d6c889fab 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala
@@ -71,7 +71,7 @@ object DatasetExample {
val conf = new SparkConf().setAppName(s"DatasetExample with $params")
val sc = new SparkContext(conf)
val sqlContext = new SQLContext(sc)
- import sqlContext._ // for implicit conversions
+ import sqlContext.implicits._ // for implicit conversions
// Load input data
val origData: RDD[LabeledPoint] = params.dataFormat match {
@@ -81,18 +81,18 @@ object DatasetExample {
println(s"Loaded ${origData.count()} instances from file: ${params.input}")
// Convert input data to DataFrame explicitly.
- val df: DataFrame = origData.toDataFrame
+ val df: DataFrame = origData.toDF()
println(s"Inferred schema:\n${df.schema.prettyJson}")
println(s"Converted to DataFrame with ${df.count()} records")
- // Select columns, using implicit conversion to DataFrames.
- val labelsDf: DataFrame = origData.select("label")
+ // Select columns
+ val labelsDf: DataFrame = df.select("label")
val labels: RDD[Double] = labelsDf.map { case Row(v: Double) => v }
val numLabels = labels.count()
val meanLabel = labels.fold(0.0)(_ + _) / numLabels
println(s"Selected label column with average value $meanLabel")
- val featuresDf: DataFrame = origData.select("features")
+ val featuresDf: DataFrame = df.select("features")
val features: RDD[Vector] = featuresDf.map { case Row(v: Vector) => v }
val featureSummary = features.aggregate(new MultivariateOnlineSummarizer())(
(summary, feat) => summary.add(feat),
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala
index 205d80dd02682..262fd2c9611d0 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala
@@ -272,6 +272,8 @@ object DecisionTreeRunner {
case Variance => impurity.Variance
}
+ params.checkpointDir.foreach(sc.setCheckpointDir)
+
val strategy
= new Strategy(
algo = params.algo,
@@ -282,7 +284,6 @@ object DecisionTreeRunner {
minInstancesPerNode = params.minInstancesPerNode,
minInfoGain = params.minInfoGain,
useNodeIdCache = params.useNodeIdCache,
- checkpointDir = params.checkpointDir,
checkpointInterval = params.checkpointInterval)
if (params.numTrees == 1) {
val startTime = System.nanoTime()
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala
index 11e35598baf50..14cc5cbb679c5 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala
@@ -56,7 +56,7 @@ object DenseKMeans {
.text(s"number of clusters, required")
.action((x, c) => c.copy(k = x))
opt[Int]("numIterations")
- .text(s"number of iterations, default; ${defaultParams.numIterations}")
+ .text(s"number of iterations, default: ${defaultParams.numIterations}")
.action((x, c) => c.copy(numIterations = x))
opt[String]("initMode")
.text(s"initialization mode (${InitializationMode.values.mkString(",")}), " +
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala
new file mode 100644
index 0000000000000..13f24a1e59610
--- /dev/null
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.mllib
+
+import scopt.OptionParser
+
+import org.apache.spark.mllib.fpm.FPGrowth
+import org.apache.spark.{SparkConf, SparkContext}
+
+/**
+ * Example for mining frequent itemsets using FP-growth.
+ * Example usage: ./bin/run-example mllib.FPGrowthExample \
+ * --minSupport 0.8 --numPartition 2 ./data/mllib/sample_fpgrowth.txt
+ */
+object FPGrowthExample {
+
+ case class Params(
+ input: String = null,
+ minSupport: Double = 0.3,
+ numPartition: Int = -1) extends AbstractParams[Params]
+
+ def main(args: Array[String]) {
+ val defaultParams = Params()
+
+ val parser = new OptionParser[Params]("FPGrowthExample") {
+ head("FPGrowth: an example FP-growth app.")
+ opt[Double]("minSupport")
+ .text(s"minimal support level, default: ${defaultParams.minSupport}")
+ .action((x, c) => c.copy(minSupport = x))
+ opt[Int]("numPartition")
+ .text(s"number of partition, default: ${defaultParams.numPartition}")
+ .action((x, c) => c.copy(numPartition = x))
+ arg[String]("")
+ .text("input paths to input data set, whose file format is that each line " +
+ "contains a transaction with each item in String and separated by a space")
+ .required()
+ .action((x, c) => c.copy(input = x))
+ }
+
+ parser.parse(args, defaultParams).map { params =>
+ run(params)
+ }.getOrElse {
+ sys.exit(1)
+ }
+ }
+
+ def run(params: Params) {
+ val conf = new SparkConf().setAppName(s"FPGrowthExample with $params")
+ val sc = new SparkContext(conf)
+ val transactions = sc.textFile(params.input).map(_.split(" ")).cache()
+
+ println(s"Number of transactions: ${transactions.count()}")
+
+ val model = new FPGrowth()
+ .setMinSupport(params.minSupport)
+ .setNumPartitions(params.numPartition)
+ .run(transactions)
+
+ println(s"Number of frequent itemsets: ${model.freqItemsets.count()}")
+
+ model.freqItemsets.collect().foreach { itemset =>
+ println(itemset.items.mkString("[", ",", "]") + ", " + itemset.freq)
+ }
+
+ sc.stop()
+ }
+}
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala
new file mode 100644
index 0000000000000..08a93595a2e17
--- /dev/null
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.mllib
+
+import java.text.BreakIterator
+
+import scala.collection.mutable
+
+import scopt.OptionParser
+
+import org.apache.log4j.{Level, Logger}
+
+import org.apache.spark.{SparkContext, SparkConf}
+import org.apache.spark.mllib.clustering.LDA
+import org.apache.spark.mllib.linalg.{Vector, Vectors}
+import org.apache.spark.rdd.RDD
+
+
+/**
+ * An example Latent Dirichlet Allocation (LDA) app. Run with
+ * {{{
+ * ./bin/run-example mllib.LDAExample [options]
+ * }}}
+ * If you use it as a template to create your own app, please use `spark-submit` to submit your app.
+ */
+object LDAExample {
+
+ private case class Params(
+ input: Seq[String] = Seq.empty,
+ k: Int = 20,
+ maxIterations: Int = 10,
+ docConcentration: Double = -1,
+ topicConcentration: Double = -1,
+ vocabSize: Int = 10000,
+ stopwordFile: String = "",
+ checkpointDir: Option[String] = None,
+ checkpointInterval: Int = 10) extends AbstractParams[Params]
+
+ def main(args: Array[String]) {
+ val defaultParams = Params()
+
+ val parser = new OptionParser[Params]("LDAExample") {
+ head("LDAExample: an example LDA app for plain text data.")
+ opt[Int]("k")
+ .text(s"number of topics. default: ${defaultParams.k}")
+ .action((x, c) => c.copy(k = x))
+ opt[Int]("maxIterations")
+ .text(s"number of iterations of learning. default: ${defaultParams.maxIterations}")
+ .action((x, c) => c.copy(maxIterations = x))
+ opt[Double]("docConcentration")
+ .text(s"amount of topic smoothing to use (> 1.0) (-1=auto)." +
+ s" default: ${defaultParams.docConcentration}")
+ .action((x, c) => c.copy(docConcentration = x))
+ opt[Double]("topicConcentration")
+ .text(s"amount of term (word) smoothing to use (> 1.0) (-1=auto)." +
+ s" default: ${defaultParams.topicConcentration}")
+ .action((x, c) => c.copy(topicConcentration = x))
+ opt[Int]("vocabSize")
+ .text(s"number of distinct word types to use, chosen by frequency. (-1=all)" +
+ s" default: ${defaultParams.vocabSize}")
+ .action((x, c) => c.copy(vocabSize = x))
+ opt[String]("stopwordFile")
+ .text(s"filepath for a list of stopwords. Note: This must fit on a single machine." +
+ s" default: ${defaultParams.stopwordFile}")
+ .action((x, c) => c.copy(stopwordFile = x))
+ opt[String]("checkpointDir")
+ .text(s"Directory for checkpointing intermediate results." +
+ s" Checkpointing helps with recovery and eliminates temporary shuffle files on disk." +
+ s" default: ${defaultParams.checkpointDir}")
+ .action((x, c) => c.copy(checkpointDir = Some(x)))
+ opt[Int]("checkpointInterval")
+ .text(s"Iterations between each checkpoint. Only used if checkpointDir is set." +
+ s" default: ${defaultParams.checkpointInterval}")
+ .action((x, c) => c.copy(checkpointInterval = x))
+ arg[String]("...")
+ .text("input paths (directories) to plain text corpora." +
+ " Each text file line should hold 1 document.")
+ .unbounded()
+ .required()
+ .action((x, c) => c.copy(input = c.input :+ x))
+ }
+
+ parser.parse(args, defaultParams).map { params =>
+ run(params)
+ }.getOrElse {
+ parser.showUsageAsError
+ sys.exit(1)
+ }
+ }
+
+ private def run(params: Params) {
+ val conf = new SparkConf().setAppName(s"LDAExample with $params")
+ val sc = new SparkContext(conf)
+
+ Logger.getRootLogger.setLevel(Level.WARN)
+
+ // Load documents, and prepare them for LDA.
+ val preprocessStart = System.nanoTime()
+ val (corpus, vocabArray, actualNumTokens) =
+ preprocess(sc, params.input, params.vocabSize, params.stopwordFile)
+ corpus.cache()
+ val actualCorpusSize = corpus.count()
+ val actualVocabSize = vocabArray.size
+ val preprocessElapsed = (System.nanoTime() - preprocessStart) / 1e9
+
+ println()
+ println(s"Corpus summary:")
+ println(s"\t Training set size: $actualCorpusSize documents")
+ println(s"\t Vocabulary size: $actualVocabSize terms")
+ println(s"\t Training set size: $actualNumTokens tokens")
+ println(s"\t Preprocessing time: $preprocessElapsed sec")
+ println()
+
+ // Run LDA.
+ val lda = new LDA()
+ lda.setK(params.k)
+ .setMaxIterations(params.maxIterations)
+ .setDocConcentration(params.docConcentration)
+ .setTopicConcentration(params.topicConcentration)
+ .setCheckpointInterval(params.checkpointInterval)
+ if (params.checkpointDir.nonEmpty) {
+ sc.setCheckpointDir(params.checkpointDir.get)
+ }
+ val startTime = System.nanoTime()
+ val ldaModel = lda.run(corpus)
+ val elapsed = (System.nanoTime() - startTime) / 1e9
+
+ println(s"Finished training LDA model. Summary:")
+ println(s"\t Training time: $elapsed sec")
+ val avgLogLikelihood = ldaModel.logLikelihood / actualCorpusSize.toDouble
+ println(s"\t Training data average log likelihood: $avgLogLikelihood")
+ println()
+
+ // Print the topics, showing the top-weighted terms for each topic.
+ val topicIndices = ldaModel.describeTopics(maxTermsPerTopic = 10)
+ val topics = topicIndices.map { case (terms, termWeights) =>
+ terms.zip(termWeights).map { case (term, weight) => (vocabArray(term.toInt), weight) }
+ }
+ println(s"${params.k} topics:")
+ topics.zipWithIndex.foreach { case (topic, i) =>
+ println(s"TOPIC $i")
+ topic.foreach { case (term, weight) =>
+ println(s"$term\t$weight")
+ }
+ println()
+ }
+ sc.stop()
+ }
+
+ /**
+ * Load documents, tokenize them, create vocabulary, and prepare documents as term count vectors.
+ * @return (corpus, vocabulary as array, total token count in corpus)
+ */
+ private def preprocess(
+ sc: SparkContext,
+ paths: Seq[String],
+ vocabSize: Int,
+ stopwordFile: String): (RDD[(Long, Vector)], Array[String], Long) = {
+
+ // Get dataset of document texts
+ // One document per line in each text file. If the input consists of many small files,
+ // this can result in a large number of small partitions, which can degrade performance.
+ // In this case, consider using coalesce() to create fewer, larger partitions.
+ val textRDD: RDD[String] = sc.textFile(paths.mkString(","))
+
+ // Split text into words
+ val tokenizer = new SimpleTokenizer(sc, stopwordFile)
+ val tokenized: RDD[(Long, IndexedSeq[String])] = textRDD.zipWithIndex().map { case (text, id) =>
+ id -> tokenizer.getWords(text)
+ }
+ tokenized.cache()
+
+ // Counts words: RDD[(word, wordCount)]
+ val wordCounts: RDD[(String, Long)] = tokenized
+ .flatMap { case (_, tokens) => tokens.map(_ -> 1L) }
+ .reduceByKey(_ + _)
+ wordCounts.cache()
+ val fullVocabSize = wordCounts.count()
+ // Select vocab
+ // (vocab: Map[word -> id], total tokens after selecting vocab)
+ val (vocab: Map[String, Int], selectedTokenCount: Long) = {
+ val tmpSortedWC: Array[(String, Long)] = if (vocabSize == -1 || fullVocabSize <= vocabSize) {
+ // Use all terms
+ wordCounts.collect().sortBy(-_._2)
+ } else {
+ // Sort terms to select vocab
+ wordCounts.sortBy(_._2, ascending = false).take(vocabSize)
+ }
+ (tmpSortedWC.map(_._1).zipWithIndex.toMap, tmpSortedWC.map(_._2).sum)
+ }
+
+ val documents = tokenized.map { case (id, tokens) =>
+ // Filter tokens by vocabulary, and create word count vector representation of document.
+ val wc = new mutable.HashMap[Int, Int]()
+ tokens.foreach { term =>
+ if (vocab.contains(term)) {
+ val termIndex = vocab(term)
+ wc(termIndex) = wc.getOrElse(termIndex, 0) + 1
+ }
+ }
+ val indices = wc.keys.toArray.sorted
+ val values = indices.map(i => wc(i).toDouble)
+
+ val sb = Vectors.sparse(vocab.size, indices, values)
+ (id, sb)
+ }
+
+ val vocabArray = new Array[String](vocab.size)
+ vocab.foreach { case (term, i) => vocabArray(i) = term }
+
+ (documents, vocabArray, selectedTokenCount)
+ }
+}
+
+/**
+ * Simple Tokenizer.
+ *
+ * TODO: Formalize the interface, and make this a public class in mllib.feature
+ */
+private class SimpleTokenizer(sc: SparkContext, stopwordFile: String) extends Serializable {
+
+ private val stopwords: Set[String] = if (stopwordFile.isEmpty) {
+ Set.empty[String]
+ } else {
+ val stopwordText = sc.textFile(stopwordFile).collect()
+ stopwordText.flatMap(_.stripMargin.split("\\s+")).toSet
+ }
+
+ // Matches sequences of Unicode letters
+ private val allWordRegex = "^(\\p{L}*)$".r
+
+ // Ignore words shorter than this length.
+ private val minWordLength = 3
+
+ def getWords(text: String): IndexedSeq[String] = {
+
+ val words = new mutable.ArrayBuffer[String]()
+
+ // Use Java BreakIterator to tokenize text into words.
+ val wb = BreakIterator.getWordInstance
+ wb.setText(text)
+
+ // current,end index start,end of each word
+ var current = wb.first()
+ var end = wb.next()
+ while (end != BreakIterator.DONE) {
+ // Convert to lowercase
+ val word: String = text.substring(current, end).toLowerCase
+ // Remove short words and strings that aren't only letters
+ word match {
+ case allWordRegex(w) if w.length >= minWordLength && !stopwords.contains(w) =>
+ words += w
+ case _ =>
+ }
+
+ current = end
+ try {
+ end = wb.next()
+ } catch {
+ case e: Exception =>
+ // Ignore remaining text in line.
+ // This is a known bug in BreakIterator (for some Java versions),
+ // which fails when it sees certain characters.
+ end = BreakIterator.DONE
+ }
+ }
+ words
+ }
+
+}
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala
new file mode 100644
index 0000000000000..91c9772744f18
--- /dev/null
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.mllib
+
+import org.apache.log4j.{Level, Logger}
+import scopt.OptionParser
+
+import org.apache.spark.mllib.clustering.PowerIterationClustering
+import org.apache.spark.rdd.RDD
+import org.apache.spark.{SparkConf, SparkContext}
+
+/**
+ * An example Power Iteration Clustering http://www.icml2010.org/papers/387.pdf app.
+ * Takes an input of K concentric circles and the number of points in the innermost circle.
+ * The output should be K clusters - each cluster containing precisely the points associated
+ * with each of the input circles.
+ *
+ * Run with
+ * {{{
+ * ./bin/run-example mllib.PowerIterationClusteringExample [options]
+ *
+ * Where options include:
+ * k: Number of circles/clusters
+ * n: Number of sampled points on innermost circle.. There are proportionally more points
+ * within the outer/larger circles
+ * maxIterations: Number of Power Iterations
+ * outerRadius: radius of the outermost of the concentric circles
+ * }}}
+ *
+ * Here is a sample run and output:
+ *
+ * ./bin/run-example mllib.PowerIterationClusteringExample -k 3 --n 30 --maxIterations 15
+ *
+ * Cluster assignments: 1 -> [0,1,2,3,4],2 -> [5,6,7,8,9,10,11,12,13,14],
+ * 0 -> [15,16,17,18,19,20,21,22,23,24,25,26,27,28,29]
+ *
+ *
+ * If you use it as a template to create your own app, please use `spark-submit` to submit your app.
+ */
+object PowerIterationClusteringExample {
+
+ case class Params(
+ input: String = null,
+ k: Int = 3,
+ numPoints: Int = 5,
+ maxIterations: Int = 10,
+ outerRadius: Double = 3.0
+ ) extends AbstractParams[Params]
+
+ def main(args: Array[String]) {
+ val defaultParams = Params()
+
+ val parser = new OptionParser[Params]("PIC Circles") {
+ head("PowerIterationClusteringExample: an example PIC app using concentric circles.")
+ opt[Int]('k', "k")
+ .text(s"number of circles (/clusters), default: ${defaultParams.k}")
+ .action((x, c) => c.copy(k = x))
+ opt[Int]('n', "n")
+ .text(s"number of points in smallest circle, default: ${defaultParams.numPoints}")
+ .action((x, c) => c.copy(numPoints = x))
+ opt[Int]("maxIterations")
+ .text(s"number of iterations, default: ${defaultParams.maxIterations}")
+ .action((x, c) => c.copy(maxIterations = x))
+ opt[Int]('r', "r")
+ .text(s"radius of outermost circle, default: ${defaultParams.outerRadius}")
+ .action((x, c) => c.copy(numPoints = x))
+ }
+
+ parser.parse(args, defaultParams).map { params =>
+ run(params)
+ }.getOrElse {
+ sys.exit(1)
+ }
+ }
+
+ def run(params: Params) {
+ val conf = new SparkConf()
+ .setMaster("local")
+ .setAppName(s"PowerIterationClustering with $params")
+ val sc = new SparkContext(conf)
+
+ Logger.getRootLogger.setLevel(Level.WARN)
+
+ val circlesRdd = generateCirclesRdd(sc, params.k, params.numPoints, params.outerRadius)
+ val model = new PowerIterationClustering()
+ .setK(params.k)
+ .setMaxIterations(params.maxIterations)
+ .run(circlesRdd)
+
+ val clusters = model.assignments.collect().groupBy(_.cluster).mapValues(_.map(_.id))
+ val assignments = clusters.toList.sortBy { case (k, v) => v.length}
+ val assignmentsStr = assignments
+ .map { case (k, v) =>
+ s"$k -> ${v.sorted.mkString("[", ",", "]")}"
+ }.mkString(",")
+ val sizesStr = assignments.map {
+ _._2.size
+ }.sorted.mkString("(", ",", ")")
+ println(s"Cluster assignments: $assignmentsStr\ncluster sizes: $sizesStr")
+
+ sc.stop()
+ }
+
+ def generateCircle(radius: Double, n: Int) = {
+ Seq.tabulate(n) { i =>
+ val theta = 2.0 * math.Pi * i / n
+ (radius * math.cos(theta), radius * math.sin(theta))
+ }
+ }
+
+ def generateCirclesRdd(sc: SparkContext,
+ nCircles: Int = 3,
+ nPoints: Int = 30,
+ outerRadius: Double): RDD[(Long, Long, Double)] = {
+
+ val radii = Array.tabulate(nCircles) { cx => outerRadius / (nCircles - cx)}
+ val groupSizes = Array.tabulate(nCircles) { cx => (cx + 1) * nPoints}
+ val points = (0 until nCircles).flatMap { cx =>
+ generateCircle(radii(cx), groupSizes(cx))
+ }.zipWithIndex
+ val rdd = sc.parallelize(points)
+ val distancesRdd = rdd.cartesian(rdd).flatMap { case (((x0, y0), i0), ((x1, y1), i1)) =>
+ if (i0 < i1) {
+ Some((i0.toLong, i1.toLong, gaussianSimilarity((x0, y0), (x1, y1), 1.0)))
+ } else {
+ None
+ }
+ }
+ distancesRdd
+ }
+
+ /**
+ * Gaussian Similarity: http://en.wikipedia.org/wiki/Radial_basis_function_kernel
+ */
+ def gaussianSimilarity(p1: (Double, Double), p2: (Double, Double), sigma: Double) = {
+ val coeff = 1.0 / (math.sqrt(2.0 * math.Pi) * sigma)
+ val expCoeff = -1.0 / 2.0 * math.pow(sigma, 2.0)
+ val ssquares = (p1._1 - p2._1) * (p1._1 - p2._1) + (p1._2 - p2._2) * (p1._2 - p2._2)
+ coeff * math.exp(expCoeff * ssquares)
+ }
+}
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala
index c5bd5b0b178d9..1a95048bbfe2d 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala
@@ -35,8 +35,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext}
*
* To run on your local machine using the two directories `trainingDir` and `testDir`,
* with updates every 5 seconds, and 2 features per data point, call:
- * $ bin/run-example \
- * org.apache.spark.examples.mllib.StreamingLinearRegression trainingDir testDir 5 2
+ * $ bin/run-example mllib.StreamingLinearRegression trainingDir testDir 5 2
*
* As you add text files to `trainingDir` the model will continuously update.
* Anytime you add text files to `testDir`, you'll see predictions from the current model.
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala
new file mode 100644
index 0000000000000..e1998099c2d78
--- /dev/null
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.mllib
+
+import org.apache.spark.mllib.linalg.Vectors
+import org.apache.spark.mllib.regression.LabeledPoint
+import org.apache.spark.mllib.classification.StreamingLogisticRegressionWithSGD
+import org.apache.spark.SparkConf
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+
+/**
+ * Train a logistic regression model on one stream of data and make predictions
+ * on another stream, where the data streams arrive as text files
+ * into two different directories.
+ *
+ * The rows of the text files must be labeled data points in the form
+ * `(y,[x1,x2,x3,...,xn])`
+ * Where n is the number of features, y is a binary label, and
+ * n must be the same for train and test.
+ *
+ * Usage: StreamingLogisticRegression
+ *
+ * To run on your local machine using the two directories `trainingDir` and `testDir`,
+ * with updates every 5 seconds, and 2 features per data point, call:
+ * $ bin/run-example mllib.StreamingLogisticRegression trainingDir testDir 5 2
+ *
+ * As you add text files to `trainingDir` the model will continuously update.
+ * Anytime you add text files to `testDir`, you'll see predictions from the current model.
+ *
+ */
+object StreamingLogisticRegression {
+
+ def main(args: Array[String]) {
+
+ if (args.length != 4) {
+ System.err.println(
+ "Usage: StreamingLogisticRegression ")
+ System.exit(1)
+ }
+
+ val conf = new SparkConf().setMaster("local").setAppName("StreamingLogisticRegression")
+ val ssc = new StreamingContext(conf, Seconds(args(2).toLong))
+
+ val trainingData = ssc.textFileStream(args(0)).map(LabeledPoint.parse)
+ val testData = ssc.textFileStream(args(1)).map(LabeledPoint.parse)
+
+ val model = new StreamingLogisticRegressionWithSGD()
+ .setInitialWeights(Vectors.zeros(args(3).toInt))
+
+ model.trainOn(trainingData)
+ model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print()
+
+ ssc.start()
+ ssc.awaitTermination()
+
+ }
+
+}
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
index 82a0b637b3cff..6331d1c0060f8 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
@@ -19,7 +19,7 @@ package org.apache.spark.examples.sql
import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.sql.SQLContext
-import org.apache.spark.sql.Dsl._
+import org.apache.spark.sql.functions._
// One method for defining the schema of an RDD is to make a case class with the desired column
// names and types.
@@ -32,33 +32,33 @@ object RDDRelation {
val sqlContext = new SQLContext(sc)
// Importing the SQL context gives access to all the SQL functions and implicit conversions.
- import sqlContext._
+ import sqlContext.implicits._
- val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i")))
+ val df = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))).toDF()
// Any RDD containing case classes can be registered as a table. The schema of the table is
// automatically inferred using scala reflection.
- rdd.registerTempTable("records")
+ df.registerTempTable("records")
// Once tables have been registered, you can run SQL queries over them.
println("Result of SELECT *:")
- sql("SELECT * FROM records").collect().foreach(println)
+ sqlContext.sql("SELECT * FROM records").collect().foreach(println)
// Aggregation queries are also supported.
- val count = sql("SELECT COUNT(*) FROM records").collect().head.getLong(0)
+ val count = sqlContext.sql("SELECT COUNT(*) FROM records").collect().head.getLong(0)
println(s"COUNT(*): $count")
// The results of SQL queries are themselves RDDs and support all normal RDD functions. The
// items in the RDD are of type Row, which allows you to access each column by ordinal.
- val rddFromSql = sql("SELECT key, value FROM records WHERE key < 10")
+ val rddFromSql = sqlContext.sql("SELECT key, value FROM records WHERE key < 10")
println("Result of RDD.map:")
rddFromSql.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect().foreach(println)
// Queries can also be written using a LINQ-like Scala DSL.
- rdd.where($"key" === 1).orderBy($"value".asc).select($"key").collect().foreach(println)
+ df.where($"key" === 1).orderBy($"value".asc).select($"key").collect().foreach(println)
// Write out an RDD as a parquet file.
- rdd.saveAsParquetFile("pair.parquet")
+ df.saveAsParquetFile("pair.parquet")
// Read in parquet file. Parquet files are self-describing so the schmema is preserved.
val parquetFile = sqlContext.parquetFile("pair.parquet")
@@ -68,7 +68,7 @@ object RDDRelation {
// These files can also be registered as tables.
parquetFile.registerTempTable("parquetFile")
- sql("SELECT * FROM parquetFile").collect().foreach(println)
+ sqlContext.sql("SELECT * FROM parquetFile").collect().foreach(println)
sc.stop()
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
index 5725da1848114..b7ba60ec28155 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
@@ -43,7 +43,8 @@ object HiveFromSpark {
// HiveContext. When not configured by the hive-site.xml, the context automatically
// creates metastore_db and warehouse in the current directory.
val hiveContext = new HiveContext(sc)
- import hiveContext._
+ import hiveContext.implicits._
+ import hiveContext.sql
sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
sql(s"LOAD DATA LOCAL INPATH '${kv1File.getAbsolutePath}' INTO TABLE src")
@@ -67,7 +68,7 @@ object HiveFromSpark {
// You can also register RDDs as temporary tables within a HiveContext.
val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i")))
- rdd.registerTempTable("records")
+ rdd.toDF().registerTempTable("records")
// Queries can then join RDD data with data stored in Hive.
println("Result of SELECT *:")
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala
index 6ff0c47793a25..f40caad322f59 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala
@@ -17,8 +17,8 @@
package org.apache.spark.examples.streaming
-import org.eclipse.paho.client.mqttv3.{MqttClient, MqttClientPersistence, MqttException, MqttMessage, MqttTopic}
-import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence
+import org.eclipse.paho.client.mqttv3._
+import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.{Seconds, StreamingContext}
@@ -31,8 +31,6 @@ import org.apache.spark.SparkConf
*/
object MQTTPublisher {
- var client: MqttClient = _
-
def main(args: Array[String]) {
if (args.length < 2) {
System.err.println("Usage: MQTTPublisher ")
@@ -42,25 +40,36 @@ object MQTTPublisher {
StreamingExamples.setStreamingLogLevels()
val Seq(brokerUrl, topic) = args.toSeq
+
+ var client: MqttClient = null
try {
- var peristance:MqttClientPersistence =new MqttDefaultFilePersistence("/tmp")
- client = new MqttClient(brokerUrl, MqttClient.generateClientId(), peristance)
+ val persistence = new MemoryPersistence()
+ client = new MqttClient(brokerUrl, MqttClient.generateClientId(), persistence)
+
+ client.connect()
+
+ val msgtopic = client.getTopic(topic)
+ val msgContent = "hello mqtt demo for spark streaming"
+ val message = new MqttMessage(msgContent.getBytes("utf-8"))
+
+ while (true) {
+ try {
+ msgtopic.publish(message)
+ println(s"Published data. topic: {msgtopic.getName()}; Message: {message}")
+ } catch {
+ case e: MqttException if e.getReasonCode == MqttException.REASON_CODE_MAX_INFLIGHT =>
+ Thread.sleep(10)
+ println("Queue is full, wait for to consume data from the message queue")
+ }
+ }
} catch {
case e: MqttException => println("Exception Caught: " + e)
+ } finally {
+ if (client != null) {
+ client.disconnect()
+ }
}
-
- client.connect()
-
- val msgtopic: MqttTopic = client.getTopic(topic)
- val msg: String = "hello mqtt demo for spark streaming"
-
- while (true) {
- val message: MqttMessage = new MqttMessage(String.valueOf(msg).getBytes("utf-8"))
- msgtopic.publish(message)
- println("Published data. topic: " + msgtopic.getName() + " Message: " + message)
- }
- client.disconnect()
}
}
@@ -96,9 +105,9 @@ object MQTTWordCount {
val sparkConf = new SparkConf().setAppName("MQTTWordCount")
val ssc = new StreamingContext(sparkConf, Seconds(2))
val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_ONLY_SER_2)
-
- val words = lines.flatMap(x => x.toString.split(" "))
+ val words = lines.flatMap(x => x.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
+
wordCounts.print()
ssc.start()
ssc.awaitTermination()
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala
new file mode 100644
index 0000000000000..5a6b9216a3fbc
--- /dev/null
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.streaming
+
+import org.apache.spark.SparkConf
+import org.apache.spark.SparkContext
+import org.apache.spark.rdd.RDD
+import org.apache.spark.streaming.{Time, Seconds, StreamingContext}
+import org.apache.spark.util.IntParam
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.storage.StorageLevel
+
+/**
+ * Use DataFrames and SQL to count words in UTF8 encoded, '\n' delimited text received from the
+ * network every second.
+ *
+ * Usage: SqlNetworkWordCount
+ * and describe the TCP server that Spark Streaming would connect to receive data.
+ *
+ * To run this on your local machine, you need to first run a Netcat server
+ * `$ nc -lk 9999`
+ * and then run the example
+ * `$ bin/run-example org.apache.spark.examples.streaming.SqlNetworkWordCount localhost 9999`
+ */
+
+object SqlNetworkWordCount {
+ def main(args: Array[String]) {
+ if (args.length < 2) {
+ System.err.println("Usage: NetworkWordCount ")
+ System.exit(1)
+ }
+
+ StreamingExamples.setStreamingLogLevels()
+
+ // Create the context with a 2 second batch size
+ val sparkConf = new SparkConf().setAppName("SqlNetworkWordCount")
+ val ssc = new StreamingContext(sparkConf, Seconds(2))
+
+ // Create a socket stream on target ip:port and count the
+ // words in input stream of \n delimited text (eg. generated by 'nc')
+ // Note that no duplication in storage level only for running locally.
+ // Replication necessary in distributed scenario for fault tolerance.
+ val lines = ssc.socketTextStream(args(0), args(1).toInt, StorageLevel.MEMORY_AND_DISK_SER)
+ val words = lines.flatMap(_.split(" "))
+
+ // Convert RDDs of the words DStream to DataFrame and run SQL query
+ words.foreachRDD((rdd: RDD[String], time: Time) => {
+ // Get the singleton instance of SQLContext
+ val sqlContext = SQLContextSingleton.getInstance(rdd.sparkContext)
+ import sqlContext.implicits._
+
+ // Convert RDD[String] to RDD[case class] to DataFrame
+ val wordsDataFrame = rdd.map(w => Record(w)).toDF()
+
+ // Register as table
+ wordsDataFrame.registerTempTable("words")
+
+ // Do word count on table using SQL and print it
+ val wordCountsDataFrame =
+ sqlContext.sql("select word, count(*) as total from words group by word")
+ println(s"========= $time =========")
+ wordCountsDataFrame.show()
+ })
+
+ ssc.start()
+ ssc.awaitTermination()
+ }
+}
+
+
+/** Case class for converting RDD to DataFrame */
+case class Record(word: String)
+
+
+/** Lazily instantiated singleton instance of SQLContext */
+object SQLContextSingleton {
+
+ @transient private var instance: SQLContext = _
+
+ def getInstance(sparkContext: SparkContext): SQLContext = {
+ if (instance == null) {
+ instance = new SQLContext(sparkContext)
+ }
+ instance
+ }
+}
diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml
index 0706f1ebf66e2..96c2787e35cd0 100644
--- a/external/flume-sink/pom.xml
+++ b/external/flume-sink/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/external/flume/pom.xml b/external/flume/pom.xml
index 1f2681394c583..172d447b77cda 100644
--- a/external/flume/pom.xml
+++ b/external/flume/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala
index 4b732c1592ab2..44dec45c227ca 100644
--- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala
+++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala
@@ -19,7 +19,6 @@ package org.apache.spark.streaming.flume
import java.net.InetSocketAddress
-import org.apache.spark.annotation.Experimental
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.StreamingContext
import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext}
@@ -121,7 +120,6 @@ object FlumeUtils {
* @param port Port of the host at which the Spark Sink is listening
* @param storageLevel Storage level to use for storing the received objects
*/
- @Experimental
def createPollingStream(
ssc: StreamingContext,
hostname: String,
@@ -138,7 +136,6 @@ object FlumeUtils {
* @param addresses List of InetSocketAddresses representing the hosts to connect to.
* @param storageLevel Storage level to use for storing the received objects
*/
- @Experimental
def createPollingStream(
ssc: StreamingContext,
addresses: Seq[InetSocketAddress],
@@ -159,7 +156,6 @@ object FlumeUtils {
* result in this stream using more threads
* @param storageLevel Storage level to use for storing the received objects
*/
- @Experimental
def createPollingStream(
ssc: StreamingContext,
addresses: Seq[InetSocketAddress],
@@ -178,7 +174,6 @@ object FlumeUtils {
* @param hostname Hostname of the host on which the Spark Sink is running
* @param port Port of the host at which the Spark Sink is listening
*/
- @Experimental
def createPollingStream(
jssc: JavaStreamingContext,
hostname: String,
@@ -195,7 +190,6 @@ object FlumeUtils {
* @param port Port of the host at which the Spark Sink is listening
* @param storageLevel Storage level to use for storing the received objects
*/
- @Experimental
def createPollingStream(
jssc: JavaStreamingContext,
hostname: String,
@@ -212,7 +206,6 @@ object FlumeUtils {
* @param addresses List of InetSocketAddresses on which the Spark Sink is running.
* @param storageLevel Storage level to use for storing the received objects
*/
- @Experimental
def createPollingStream(
jssc: JavaStreamingContext,
addresses: Array[InetSocketAddress],
@@ -233,7 +226,6 @@ object FlumeUtils {
* result in this stream using more threads
* @param storageLevel Storage level to use for storing the received objects
*/
- @Experimental
def createPollingStream(
jssc: JavaStreamingContext,
addresses: Array[InetSocketAddress],
diff --git a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
index 1e24da7f5f60c..cfedb5a042a35 100644
--- a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
+++ b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
@@ -31,7 +31,7 @@ public void setUp() {
SparkConf conf = new SparkConf()
.setMaster("local[2]")
.setAppName("test")
- .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
+ .set("spark.streaming.clock", "org.apache.spark.util.ManualClock");
ssc = new JavaStreamingContext(conf, new Duration(1000));
ssc.checkpoint("checkpoint");
}
diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala
index b57a1c71e35b9..e04d4088df7dc 100644
--- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala
+++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala
@@ -34,10 +34,9 @@ import org.scalatest.{BeforeAndAfter, FunSuite}
import org.apache.spark.{SparkConf, Logging}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.dstream.ReceiverInputDStream
-import org.apache.spark.streaming.util.ManualClock
import org.apache.spark.streaming.{Seconds, TestOutputStream, StreamingContext}
import org.apache.spark.streaming.flume.sink._
-import org.apache.spark.util.Utils
+import org.apache.spark.util.{ManualClock, Utils}
class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging {
@@ -54,7 +53,7 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging
def beforeFunction() {
logInfo("Using manual clock")
- conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+ conf.set("spark.streaming.clock", "org.apache.spark.util.ManualClock")
}
before(beforeFunction())
@@ -236,7 +235,7 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging
tx.commit()
tx.close()
Thread.sleep(500) // Allow some time for the events to reach
- clock.addToTime(batchDuration.milliseconds)
+ clock.advance(batchDuration.milliseconds)
}
null
}
diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
index f333e3891b5f0..322de7bf2fed8 100644
--- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
+++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
@@ -19,13 +19,13 @@ package org.apache.spark.streaming.flume
import java.net.{InetSocketAddress, ServerSocket}
import java.nio.ByteBuffer
-import java.nio.charset.Charset
import scala.collection.JavaConversions._
import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer}
import scala.concurrent.duration._
import scala.language.postfixOps
+import com.google.common.base.Charsets
import org.apache.avro.ipc.NettyTransceiver
import org.apache.avro.ipc.specific.SpecificRequestor
import org.apache.flume.source.avro
@@ -108,7 +108,7 @@ class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with L
val inputEvents = input.map { item =>
val event = new AvroFlumeEvent
- event.setBody(ByteBuffer.wrap(item.getBytes("UTF-8")))
+ event.setBody(ByteBuffer.wrap(item.getBytes(Charsets.UTF_8)))
event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header"))
event
}
@@ -138,14 +138,13 @@ class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with L
status should be (avro.Status.OK)
}
- val decoder = Charset.forName("UTF-8").newDecoder()
eventually(timeout(10 seconds), interval(100 milliseconds)) {
val outputEvents = outputBuffer.flatten.map { _.event }
outputEvents.foreach {
event =>
event.getHeaders.get("test") should be("header")
}
- val output = outputEvents.map(event => decoder.decode(event.getBody()).toString)
+ val output = outputEvents.map(event => new String(event.getBody.array(), Charsets.UTF_8))
output should be (input)
}
}
diff --git a/external/kafka-assembly/pom.xml b/external/kafka-assembly/pom.xml
new file mode 100644
index 0000000000000..5109b8ed87524
--- /dev/null
+++ b/external/kafka-assembly/pom.xml
@@ -0,0 +1,102 @@
+
+
+
+
+ 4.0.0
+
+ org.apache.spark
+ spark-parent_2.10
+ 1.3.0-SNAPSHOT
+ ../../pom.xml
+
+
+ org.apache.spark
+ spark-streaming-kafka-assembly_2.10
+ jar
+ Spark Project External Kafka Assembly
+ http://spark.apache.org/
+
+
+ streaming-kafka-assembly
+
+
+
+
+ org.apache.spark
+ spark-streaming-kafka_${scala.binary.version}
+ ${project.version}
+
+
+ org.apache.spark
+ spark-streaming_${scala.binary.version}
+ ${project.version}
+ provided
+
+
+
+
+ target/scala-${scala.binary.version}/classes
+ target/scala-${scala.binary.version}/test-classes
+
+
+ org.apache.maven.plugins
+ maven-shade-plugin
+
+ false
+
+
+ *:*
+
+
+
+
+ *:*
+
+ META-INF/*.SF
+ META-INF/*.DSA
+ META-INF/*.RSA
+
+
+
+
+
+
+ package
+
+ shade
+
+
+
+
+
+ reference.conf
+
+
+ log4j.properties
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml
index b29b0509656ba..369856187a244 100644
--- a/external/kafka/pom.xml
+++ b/external/kafka/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
@@ -44,7 +44,7 @@
org.apache.kafka
kafka_${scala.binary.version}
- 0.8.0
+ 0.8.1.1
com.sun.jmx
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala
new file mode 100644
index 0000000000000..5a74febb4bd46
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala
@@ -0,0 +1,68 @@
+/*
+ * 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.streaming.kafka
+
+import org.apache.spark.annotation.Experimental
+
+/**
+ * :: Experimental ::
+ * Represent the host and port info for a Kafka broker.
+ * Differs from the Kafka project's internal kafka.cluster.Broker, which contains a server ID
+ */
+@Experimental
+final class Broker private(
+ /** Broker's hostname */
+ val host: String,
+ /** Broker's port */
+ val port: Int) extends Serializable {
+ override def equals(obj: Any): Boolean = obj match {
+ case that: Broker =>
+ this.host == that.host &&
+ this.port == that.port
+ case _ => false
+ }
+
+ override def hashCode: Int = {
+ 41 * (41 + host.hashCode) + port
+ }
+
+ override def toString(): String = {
+ s"Broker($host, $port)"
+ }
+}
+
+/**
+ * :: Experimental ::
+ * Companion object that provides methods to create instances of [[Broker]].
+ */
+@Experimental
+object Broker {
+ def create(host: String, port: Int): Broker =
+ new Broker(host, port)
+
+ def apply(host: String, port: Int): Broker =
+ new Broker(host, port)
+
+ def unapply(broker: Broker): Option[(String, Int)] = {
+ if (broker == null) {
+ None
+ } else {
+ Some((broker.host, broker.port))
+ }
+ }
+}
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala
new file mode 100644
index 0000000000000..04e65cb3d708c
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala
@@ -0,0 +1,161 @@
+/*
+ * 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.streaming.kafka
+
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.reflect.{classTag, ClassTag}
+
+import kafka.common.TopicAndPartition
+import kafka.message.MessageAndMetadata
+import kafka.serializer.Decoder
+
+import org.apache.spark.{Logging, SparkException}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset
+import org.apache.spark.streaming.{StreamingContext, Time}
+import org.apache.spark.streaming.dstream._
+
+/**
+ * A stream of {@link org.apache.spark.streaming.kafka.KafkaRDD} where
+ * each given Kafka topic/partition corresponds to an RDD partition.
+ * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number
+ * of messages
+ * per second that each '''partition''' will accept.
+ * Starting offsets are specified in advance,
+ * and this DStream is not responsible for committing offsets,
+ * so that you can control exactly-once semantics.
+ * For an easy interface to Kafka-managed offsets,
+ * see {@link org.apache.spark.streaming.kafka.KafkaCluster}
+ * @param kafkaParams Kafka
+ * configuration parameters.
+ * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s),
+ * NOT zookeeper servers, specified in host1:port1,host2:port2 form.
+ * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
+ * starting point of the stream
+ * @param messageHandler function for translating each message into the desired type
+ */
+private[streaming]
+class DirectKafkaInputDStream[
+ K: ClassTag,
+ V: ClassTag,
+ U <: Decoder[K]: ClassTag,
+ T <: Decoder[V]: ClassTag,
+ R: ClassTag](
+ @transient ssc_ : StreamingContext,
+ val kafkaParams: Map[String, String],
+ val fromOffsets: Map[TopicAndPartition, Long],
+ messageHandler: MessageAndMetadata[K, V] => R
+) extends InputDStream[R](ssc_) with Logging {
+ val maxRetries = context.sparkContext.getConf.getInt(
+ "spark.streaming.kafka.maxRetries", 1)
+
+ protected[streaming] override val checkpointData =
+ new DirectKafkaInputDStreamCheckpointData
+
+ protected val kc = new KafkaCluster(kafkaParams)
+
+ protected val maxMessagesPerPartition: Option[Long] = {
+ val ratePerSec = context.sparkContext.getConf.getInt(
+ "spark.streaming.kafka.maxRatePerPartition", 0)
+ if (ratePerSec > 0) {
+ val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000
+ Some((secsPerBatch * ratePerSec).toLong)
+ } else {
+ None
+ }
+ }
+
+ protected var currentOffsets = fromOffsets
+
+ @tailrec
+ protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = {
+ val o = kc.getLatestLeaderOffsets(currentOffsets.keySet)
+ // Either.fold would confuse @tailrec, do it manually
+ if (o.isLeft) {
+ val err = o.left.get.toString
+ if (retries <= 0) {
+ throw new SparkException(err)
+ } else {
+ log.error(err)
+ Thread.sleep(kc.config.refreshLeaderBackoffMs)
+ latestLeaderOffsets(retries - 1)
+ }
+ } else {
+ o.right.get
+ }
+ }
+
+ // limits the maximum number of messages per partition
+ protected def clamp(
+ leaderOffsets: Map[TopicAndPartition, LeaderOffset]): Map[TopicAndPartition, LeaderOffset] = {
+ maxMessagesPerPartition.map { mmp =>
+ leaderOffsets.map { case (tp, lo) =>
+ tp -> lo.copy(offset = Math.min(currentOffsets(tp) + mmp, lo.offset))
+ }
+ }.getOrElse(leaderOffsets)
+ }
+
+ override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = {
+ val untilOffsets = clamp(latestLeaderOffsets(maxRetries))
+ val rdd = KafkaRDD[K, V, U, T, R](
+ context.sparkContext, kafkaParams, currentOffsets, untilOffsets, messageHandler)
+
+ currentOffsets = untilOffsets.map(kv => kv._1 -> kv._2.offset)
+ Some(rdd)
+ }
+
+ override def start(): Unit = {
+ }
+
+ def stop(): Unit = {
+ }
+
+ private[streaming]
+ class DirectKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) {
+ def batchForTime = data.asInstanceOf[mutable.HashMap[
+ Time, Array[OffsetRange.OffsetRangeTuple]]]
+
+ override def update(time: Time) {
+ batchForTime.clear()
+ generatedRDDs.foreach { kv =>
+ val a = kv._2.asInstanceOf[KafkaRDD[K, V, U, T, R]].offsetRanges.map(_.toTuple).toArray
+ batchForTime += kv._1 -> a
+ }
+ }
+
+ override def cleanup(time: Time) { }
+
+ override def restore() {
+ // this is assuming that the topics don't change during execution, which is true currently
+ val topics = fromOffsets.keySet
+ val leaders = kc.findLeaders(topics).fold(
+ errs => throw new SparkException(errs.mkString("\n")),
+ ok => ok
+ )
+
+ batchForTime.toSeq.sortBy(_._1)(Time.ordering).foreach { case (t, b) =>
+ logInfo(s"Restoring KafkaRDD for time $t ${b.mkString("[", ", ", "]")}")
+ generatedRDDs += t -> new KafkaRDD[K, V, U, T, R](
+ context.sparkContext, kafkaParams, b.map(OffsetRange(_)), leaders, messageHandler)
+ }
+ }
+ }
+
+}
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala
new file mode 100644
index 0000000000000..2f7e0ab39fefd
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala
@@ -0,0 +1,372 @@
+/*
+ * 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.streaming.kafka
+
+import scala.util.control.NonFatal
+import scala.util.Random
+import scala.collection.mutable.ArrayBuffer
+import java.util.Properties
+import kafka.api._
+import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
+import kafka.consumer.{ConsumerConfig, SimpleConsumer}
+import org.apache.spark.SparkException
+
+/**
+ * Convenience methods for interacting with a Kafka cluster.
+ * @param kafkaParams Kafka
+ * configuration parameters.
+ * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s),
+ * NOT zookeeper servers, specified in host1:port1,host2:port2 form
+ */
+private[spark]
+class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
+ import KafkaCluster.{Err, LeaderOffset, SimpleConsumerConfig}
+
+ // ConsumerConfig isn't serializable
+ @transient private var _config: SimpleConsumerConfig = null
+
+ def config: SimpleConsumerConfig = this.synchronized {
+ if (_config == null) {
+ _config = SimpleConsumerConfig(kafkaParams)
+ }
+ _config
+ }
+
+ def connect(host: String, port: Int): SimpleConsumer =
+ new SimpleConsumer(host, port, config.socketTimeoutMs,
+ config.socketReceiveBufferBytes, config.clientId)
+
+ def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
+ findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
+
+ // Metadata api
+ // scalastyle:off
+ // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
+ // scalastyle:on
+
+ def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
+ val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
+ 0, config.clientId, Seq(topic))
+ val errs = new Err
+ withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer =>
+ val resp: TopicMetadataResponse = consumer.send(req)
+ resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
+ tm.partitionsMetadata.find(_.partitionId == partition)
+ }.foreach { pm: PartitionMetadata =>
+ pm.leader.foreach { leader =>
+ return Right((leader.host, leader.port))
+ }
+ }
+ }
+ Left(errs)
+ }
+
+ def findLeaders(
+ topicAndPartitions: Set[TopicAndPartition]
+ ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
+ val topics = topicAndPartitions.map(_.topic)
+ val response = getPartitionMetadata(topics).right
+ val answer = response.flatMap { tms: Set[TopicMetadata] =>
+ val leaderMap = tms.flatMap { tm: TopicMetadata =>
+ tm.partitionsMetadata.flatMap { pm: PartitionMetadata =>
+ val tp = TopicAndPartition(tm.topic, pm.partitionId)
+ if (topicAndPartitions(tp)) {
+ pm.leader.map { l =>
+ tp -> (l.host -> l.port)
+ }
+ } else {
+ None
+ }
+ }
+ }.toMap
+
+ if (leaderMap.keys.size == topicAndPartitions.size) {
+ Right(leaderMap)
+ } else {
+ val missing = topicAndPartitions.diff(leaderMap.keySet)
+ val err = new Err
+ err.append(new SparkException(s"Couldn't find leaders for ${missing}"))
+ Left(err)
+ }
+ }
+ answer
+ }
+
+ def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] = {
+ getPartitionMetadata(topics).right.map { r =>
+ r.flatMap { tm: TopicMetadata =>
+ tm.partitionsMetadata.map { pm: PartitionMetadata =>
+ TopicAndPartition(tm.topic, pm.partitionId)
+ }
+ }
+ }
+ }
+
+ def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
+ val req = TopicMetadataRequest(
+ TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq)
+ val errs = new Err
+ withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer =>
+ val resp: TopicMetadataResponse = consumer.send(req)
+ // error codes here indicate missing / just created topic,
+ // repeating on a different broker wont be useful
+ return Right(resp.topicsMetadata.toSet)
+ }
+ Left(errs)
+ }
+
+ // Leader offset api
+ // scalastyle:off
+ // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI
+ // scalastyle:on
+
+ def getLatestLeaderOffsets(
+ topicAndPartitions: Set[TopicAndPartition]
+ ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
+ getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
+
+ def getEarliestLeaderOffsets(
+ topicAndPartitions: Set[TopicAndPartition]
+ ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
+ getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
+
+ def getLeaderOffsets(
+ topicAndPartitions: Set[TopicAndPartition],
+ before: Long
+ ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = {
+ getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
+ r.map { kv =>
+ // mapValues isnt serializable, see SI-7005
+ kv._1 -> kv._2.head
+ }
+ }
+ }
+
+ private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
+ m.groupBy(_._2).map { kv =>
+ kv._1 -> kv._2.keys.toSeq
+ }
+
+ def getLeaderOffsets(
+ topicAndPartitions: Set[TopicAndPartition],
+ before: Long,
+ maxNumOffsets: Int
+ ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = {
+ findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
+ val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
+ val leaders = leaderToTp.keys
+ var result = Map[TopicAndPartition, Seq[LeaderOffset]]()
+ val errs = new Err
+ withBrokers(leaders, errs) { consumer =>
+ val partitionsToGetOffsets: Seq[TopicAndPartition] =
+ leaderToTp((consumer.host, consumer.port))
+ val reqMap = partitionsToGetOffsets.map { tp: TopicAndPartition =>
+ tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
+ }.toMap
+ val req = OffsetRequest(reqMap)
+ val resp = consumer.getOffsetsBefore(req)
+ val respMap = resp.partitionErrorAndOffsets
+ partitionsToGetOffsets.foreach { tp: TopicAndPartition =>
+ respMap.get(tp).foreach { por: PartitionOffsetsResponse =>
+ if (por.error == ErrorMapping.NoError) {
+ if (por.offsets.nonEmpty) {
+ result += tp -> por.offsets.map { off =>
+ LeaderOffset(consumer.host, consumer.port, off)
+ }
+ } else {
+ errs.append(new SparkException(
+ s"Empty offsets for ${tp}, is ${before} before log beginning?"))
+ }
+ } else {
+ errs.append(ErrorMapping.exceptionFor(por.error))
+ }
+ }
+ }
+ if (result.keys.size == topicAndPartitions.size) {
+ return Right(result)
+ }
+ }
+ val missing = topicAndPartitions.diff(result.keySet)
+ errs.append(new SparkException(s"Couldn't find leader offsets for ${missing}"))
+ Left(errs)
+ }
+ }
+
+ // Consumer offset api
+ // scalastyle:off
+ // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI
+ // scalastyle:on
+
+ /** Requires Kafka >= 0.8.1.1 */
+ def getConsumerOffsets(
+ groupId: String,
+ topicAndPartitions: Set[TopicAndPartition]
+ ): Either[Err, Map[TopicAndPartition, Long]] = {
+ getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
+ r.map { kv =>
+ kv._1 -> kv._2.offset
+ }
+ }
+ }
+
+ /** Requires Kafka >= 0.8.1.1 */
+ def getConsumerOffsetMetadata(
+ groupId: String,
+ topicAndPartitions: Set[TopicAndPartition]
+ ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
+ var result = Map[TopicAndPartition, OffsetMetadataAndError]()
+ val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
+ val errs = new Err
+ withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer =>
+ val resp = consumer.fetchOffsets(req)
+ val respMap = resp.requestInfo
+ val needed = topicAndPartitions.diff(result.keySet)
+ needed.foreach { tp: TopicAndPartition =>
+ respMap.get(tp).foreach { ome: OffsetMetadataAndError =>
+ if (ome.error == ErrorMapping.NoError) {
+ result += tp -> ome
+ } else {
+ errs.append(ErrorMapping.exceptionFor(ome.error))
+ }
+ }
+ }
+ if (result.keys.size == topicAndPartitions.size) {
+ return Right(result)
+ }
+ }
+ val missing = topicAndPartitions.diff(result.keySet)
+ errs.append(new SparkException(s"Couldn't find consumer offsets for ${missing}"))
+ Left(errs)
+ }
+
+ /** Requires Kafka >= 0.8.1.1 */
+ def setConsumerOffsets(
+ groupId: String,
+ offsets: Map[TopicAndPartition, Long]
+ ): Either[Err, Map[TopicAndPartition, Short]] = {
+ setConsumerOffsetMetadata(groupId, offsets.map { kv =>
+ kv._1 -> OffsetMetadataAndError(kv._2)
+ })
+ }
+
+ /** Requires Kafka >= 0.8.1.1 */
+ def setConsumerOffsetMetadata(
+ groupId: String,
+ metadata: Map[TopicAndPartition, OffsetMetadataAndError]
+ ): Either[Err, Map[TopicAndPartition, Short]] = {
+ var result = Map[TopicAndPartition, Short]()
+ val req = OffsetCommitRequest(groupId, metadata)
+ val errs = new Err
+ val topicAndPartitions = metadata.keySet
+ withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer =>
+ val resp = consumer.commitOffsets(req)
+ val respMap = resp.requestInfo
+ val needed = topicAndPartitions.diff(result.keySet)
+ needed.foreach { tp: TopicAndPartition =>
+ respMap.get(tp).foreach { err: Short =>
+ if (err == ErrorMapping.NoError) {
+ result += tp -> err
+ } else {
+ errs.append(ErrorMapping.exceptionFor(err))
+ }
+ }
+ }
+ if (result.keys.size == topicAndPartitions.size) {
+ return Right(result)
+ }
+ }
+ val missing = topicAndPartitions.diff(result.keySet)
+ errs.append(new SparkException(s"Couldn't set offsets for ${missing}"))
+ Left(errs)
+ }
+
+ // Try a call against potentially multiple brokers, accumulating errors
+ private def withBrokers(brokers: Iterable[(String, Int)], errs: Err)
+ (fn: SimpleConsumer => Any): Unit = {
+ brokers.foreach { hp =>
+ var consumer: SimpleConsumer = null
+ try {
+ consumer = connect(hp._1, hp._2)
+ fn(consumer)
+ } catch {
+ case NonFatal(e) =>
+ errs.append(e)
+ } finally {
+ if (consumer != null) {
+ consumer.close()
+ }
+ }
+ }
+ }
+}
+
+private[spark]
+object KafkaCluster {
+ type Err = ArrayBuffer[Throwable]
+
+ private[spark]
+ case class LeaderOffset(host: String, port: Int, offset: Long)
+
+ /**
+ * High-level kafka consumers connect to ZK. ConsumerConfig assumes this use case.
+ * Simple consumers connect directly to brokers, but need many of the same configs.
+ * This subclass won't warn about missing ZK params, or presence of broker params.
+ */
+ private[spark]
+ class SimpleConsumerConfig private(brokers: String, originalProps: Properties)
+ extends ConsumerConfig(originalProps) {
+ val seedBrokers: Array[(String, Int)] = brokers.split(",").map { hp =>
+ val hpa = hp.split(":")
+ if (hpa.size == 1) {
+ throw new SparkException(s"Broker not the in correct format of : [$brokers]")
+ }
+ (hpa(0), hpa(1).toInt)
+ }
+ }
+
+ private[spark]
+ object SimpleConsumerConfig {
+ /**
+ * Make a consumer config without requiring group.id or zookeeper.connect,
+ * since communicating with brokers also needs common settings such as timeout
+ */
+ def apply(kafkaParams: Map[String, String]): SimpleConsumerConfig = {
+ // These keys are from other pre-existing kafka configs for specifying brokers, accept either
+ val brokers = kafkaParams.get("metadata.broker.list")
+ .orElse(kafkaParams.get("bootstrap.servers"))
+ .getOrElse(throw new SparkException(
+ "Must specify metadata.broker.list or bootstrap.servers"))
+
+ val props = new Properties()
+ kafkaParams.foreach { case (key, value) =>
+ // prevent warnings on parameters ConsumerConfig doesn't know about
+ if (key != "metadata.broker.list" && key != "bootstrap.servers") {
+ props.put(key, value)
+ }
+ }
+
+ Seq("zookeeper.connect", "group.id").foreach { s =>
+ if (!props.contains(s)) {
+ props.setProperty(s, "")
+ }
+ }
+
+ new SimpleConsumerConfig(brokers, props)
+ }
+ }
+}
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala
new file mode 100644
index 0000000000000..6d465bcb6bfc0
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala
@@ -0,0 +1,222 @@
+/*
+ * 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.streaming.kafka
+
+import scala.reflect.{classTag, ClassTag}
+
+import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.NextIterator
+
+import java.util.Properties
+import kafka.api.{FetchRequestBuilder, FetchResponse}
+import kafka.common.{ErrorMapping, TopicAndPartition}
+import kafka.consumer.{ConsumerConfig, SimpleConsumer}
+import kafka.message.{MessageAndMetadata, MessageAndOffset}
+import kafka.serializer.Decoder
+import kafka.utils.VerifiableProperties
+
+/**
+ * A batch-oriented interface for consuming from Kafka.
+ * Starting and ending offsets are specified in advance,
+ * so that you can control exactly-once semantics.
+ * @param kafkaParams Kafka
+ * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" to be set
+ * with Kafka broker(s) specified in host1:port1,host2:port2 form.
+ * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD
+ * @param messageHandler function for translating each message into the desired type
+ */
+private[kafka]
+class KafkaRDD[
+ K: ClassTag,
+ V: ClassTag,
+ U <: Decoder[_]: ClassTag,
+ T <: Decoder[_]: ClassTag,
+ R: ClassTag] private[spark] (
+ sc: SparkContext,
+ kafkaParams: Map[String, String],
+ val offsetRanges: Array[OffsetRange],
+ leaders: Map[TopicAndPartition, (String, Int)],
+ messageHandler: MessageAndMetadata[K, V] => R
+ ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges {
+ override def getPartitions: Array[Partition] = {
+ offsetRanges.zipWithIndex.map { case (o, i) =>
+ val (host, port) = leaders(TopicAndPartition(o.topic, o.partition))
+ new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port)
+ }.toArray
+ }
+
+ override def getPreferredLocations(thePart: Partition): Seq[String] = {
+ val part = thePart.asInstanceOf[KafkaRDDPartition]
+ // TODO is additional hostname resolution necessary here
+ Seq(part.host)
+ }
+
+ private def errBeginAfterEnd(part: KafkaRDDPartition): String =
+ s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " +
+ s"for topic ${part.topic} partition ${part.partition}. " +
+ "You either provided an invalid fromOffset, or the Kafka topic has been damaged"
+
+ private def errRanOutBeforeEnd(part: KafkaRDDPartition): String =
+ s"Ran out of messages before reaching ending offset ${part.untilOffset} " +
+ s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
+ " This should not happen, and indicates that messages may have been lost"
+
+ private def errOvershotEnd(itemOffset: Long, part: KafkaRDDPartition): String =
+ s"Got ${itemOffset} > ending offset ${part.untilOffset} " +
+ s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
+ " This should not happen, and indicates a message may have been skipped"
+
+ override def compute(thePart: Partition, context: TaskContext): Iterator[R] = {
+ val part = thePart.asInstanceOf[KafkaRDDPartition]
+ assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part))
+ if (part.fromOffset == part.untilOffset) {
+ log.warn(s"Beginning offset ${part.fromOffset} is the same as ending offset " +
+ s"skipping ${part.topic} ${part.partition}")
+ Iterator.empty
+ } else {
+ new KafkaRDDIterator(part, context)
+ }
+ }
+
+ private class KafkaRDDIterator(
+ part: KafkaRDDPartition,
+ context: TaskContext) extends NextIterator[R] {
+
+ context.addTaskCompletionListener{ context => closeIfNeeded() }
+
+ log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
+ s"offsets ${part.fromOffset} -> ${part.untilOffset}")
+
+ val kc = new KafkaCluster(kafkaParams)
+ val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
+ .newInstance(kc.config.props)
+ .asInstanceOf[Decoder[K]]
+ val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
+ .newInstance(kc.config.props)
+ .asInstanceOf[Decoder[V]]
+ val consumer = connectLeader
+ var requestOffset = part.fromOffset
+ var iter: Iterator[MessageAndOffset] = null
+
+ // The idea is to use the provided preferred host, except on task retry atttempts,
+ // to minimize number of kafka metadata requests
+ private def connectLeader: SimpleConsumer = {
+ if (context.attemptNumber > 0) {
+ kc.connectLeader(part.topic, part.partition).fold(
+ errs => throw new SparkException(
+ s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
+ errs.mkString("\n")),
+ consumer => consumer
+ )
+ } else {
+ kc.connect(part.host, part.port)
+ }
+ }
+
+ private def handleFetchErr(resp: FetchResponse) {
+ if (resp.hasError) {
+ val err = resp.errorCode(part.topic, part.partition)
+ if (err == ErrorMapping.LeaderNotAvailableCode ||
+ err == ErrorMapping.NotLeaderForPartitionCode) {
+ log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
+ s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
+ Thread.sleep(kc.config.refreshLeaderBackoffMs)
+ }
+ // Let normal rdd retry sort out reconnect attempts
+ throw ErrorMapping.exceptionFor(err)
+ }
+ }
+
+ private def fetchBatch: Iterator[MessageAndOffset] = {
+ val req = new FetchRequestBuilder()
+ .addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes)
+ .build()
+ val resp = consumer.fetch(req)
+ handleFetchErr(resp)
+ // kafka may return a batch that starts before the requested offset
+ resp.messageSet(part.topic, part.partition)
+ .iterator
+ .dropWhile(_.offset < requestOffset)
+ }
+
+ override def close() = consumer.close()
+
+ override def getNext(): R = {
+ if (iter == null || !iter.hasNext) {
+ iter = fetchBatch
+ }
+ if (!iter.hasNext) {
+ assert(requestOffset == part.untilOffset, errRanOutBeforeEnd(part))
+ finished = true
+ null.asInstanceOf[R]
+ } else {
+ val item = iter.next()
+ if (item.offset >= part.untilOffset) {
+ assert(item.offset == part.untilOffset, errOvershotEnd(item.offset, part))
+ finished = true
+ null.asInstanceOf[R]
+ } else {
+ requestOffset = item.nextOffset
+ messageHandler(new MessageAndMetadata(
+ part.topic, part.partition, item.message, item.offset, keyDecoder, valueDecoder))
+ }
+ }
+ }
+ }
+}
+
+private[kafka]
+object KafkaRDD {
+ import KafkaCluster.LeaderOffset
+
+ /**
+ * @param kafkaParams Kafka
+ * configuration parameters.
+ * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s),
+ * NOT zookeeper servers, specified in host1:port1,host2:port2 form.
+ * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
+ * starting point of the batch
+ * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive)
+ * ending point of the batch
+ * @param messageHandler function for translating each message into the desired type
+ */
+ def apply[
+ K: ClassTag,
+ V: ClassTag,
+ U <: Decoder[_]: ClassTag,
+ T <: Decoder[_]: ClassTag,
+ R: ClassTag](
+ sc: SparkContext,
+ kafkaParams: Map[String, String],
+ fromOffsets: Map[TopicAndPartition, Long],
+ untilOffsets: Map[TopicAndPartition, LeaderOffset],
+ messageHandler: MessageAndMetadata[K, V] => R
+ ): KafkaRDD[K, V, U, T, R] = {
+ val leaders = untilOffsets.map { case (tp, lo) =>
+ tp -> (lo.host, lo.port)
+ }.toMap
+
+ val offsetRanges = fromOffsets.map { case (tp, fo) =>
+ val uo = untilOffsets(tp)
+ OffsetRange(tp.topic, tp.partition, fo, uo.offset)
+ }.toArray
+
+ new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaders, messageHandler)
+ }
+}
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala
new file mode 100644
index 0000000000000..a842a6f17766f
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala
@@ -0,0 +1,38 @@
+/*
+ * 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.streaming.kafka
+
+import org.apache.spark.Partition
+
+/** @param topic kafka topic name
+ * @param partition kafka partition id
+ * @param fromOffset inclusive starting offset
+ * @param untilOffset exclusive ending offset
+ * @param host preferred kafka host, i.e. the leader at the time the rdd was created
+ * @param port preferred kafka host's port
+ */
+private[kafka]
+class KafkaRDDPartition(
+ val index: Int,
+ val topic: String,
+ val partition: Int,
+ val fromOffset: Long,
+ val untilOffset: Long,
+ val host: String,
+ val port: Int
+) extends Partition
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
index df725f0c65a64..5a9bd4214cf51 100644
--- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
@@ -18,21 +18,30 @@
package org.apache.spark.streaming.kafka
import java.lang.{Integer => JInt}
+import java.lang.{Long => JLong}
import java.util.{Map => JMap}
+import java.util.{Set => JSet}
import scala.reflect.ClassTag
import scala.collection.JavaConversions._
-import kafka.serializer.{Decoder, StringDecoder}
+import kafka.common.TopicAndPartition
+import kafka.message.MessageAndMetadata
+import kafka.serializer.{DefaultDecoder, Decoder, StringDecoder}
+import org.apache.spark.api.java.function.{Function => JFunction}
+import org.apache.spark.{SparkContext, SparkException}
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.StreamingContext
-import org.apache.spark.streaming.api.java.{JavaPairReceiverInputDStream, JavaStreamingContext}
-import org.apache.spark.streaming.dstream.ReceiverInputDStream
+import org.apache.spark.streaming.api.java.{JavaPairInputDStream, JavaInputDStream, JavaPairReceiverInputDStream, JavaStreamingContext}
+import org.apache.spark.streaming.dstream.{InputDStream, ReceiverInputDStream}
+import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD}
object KafkaUtils {
/**
- * Create an input stream that pulls messages from a Kafka Broker.
+ * Create an input stream that pulls messages from Kafka Brokers.
* @param ssc StreamingContext object
* @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..)
* @param groupId The group id for this consumer
@@ -56,7 +65,7 @@ object KafkaUtils {
}
/**
- * Create an input stream that pulls messages from a Kafka Broker.
+ * Create an input stream that pulls messages from Kafka Brokers.
* @param ssc StreamingContext object
* @param kafkaParams Map of kafka configuration parameters,
* see http://kafka.apache.org/08/configuration.html
@@ -75,7 +84,7 @@ object KafkaUtils {
}
/**
- * Create an input stream that pulls messages from a Kafka Broker.
+ * Create an input stream that pulls messages from Kafka Brokers.
* Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2.
* @param jssc JavaStreamingContext object
* @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..)
@@ -93,7 +102,7 @@ object KafkaUtils {
}
/**
- * Create an input stream that pulls messages from a Kafka Broker.
+ * Create an input stream that pulls messages from Kafka Brokers.
* @param jssc JavaStreamingContext object
* @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..).
* @param groupId The group id for this consumer.
@@ -113,10 +122,10 @@ object KafkaUtils {
}
/**
- * Create an input stream that pulls messages from a Kafka Broker.
+ * Create an input stream that pulls messages from Kafka Brokers.
* @param jssc JavaStreamingContext object
- * @param keyTypeClass Key type of RDD
- * @param valueTypeClass value type of RDD
+ * @param keyTypeClass Key type of DStream
+ * @param valueTypeClass value type of Dstream
* @param keyDecoderClass Type of kafka key decoder
* @param valueDecoderClass Type of kafka value decoder
* @param kafkaParams Map of kafka configuration parameters,
@@ -144,4 +153,409 @@ object KafkaUtils {
createStream[K, V, U, T](
jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
}
+
+ /** get leaders for the given offset ranges, or throw an exception */
+ private def leadersForRanges(
+ kafkaParams: Map[String, String],
+ offsetRanges: Array[OffsetRange]): Map[TopicAndPartition, (String, Int)] = {
+ val kc = new KafkaCluster(kafkaParams)
+ val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
+ val leaders = kc.findLeaders(topics).fold(
+ errs => throw new SparkException(errs.mkString("\n")),
+ ok => ok
+ )
+ leaders
+ }
+
+ /**
+ * Create a RDD from Kafka using offset ranges for each topic and partition.
+ *
+ * @param sc SparkContext object
+ * @param kafkaParams Kafka
+ * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers"
+ * to be set with Kafka broker(s) (NOT zookeeper servers) specified in
+ * host1:port1,host2:port2 form.
+ * @param offsetRanges Each OffsetRange in the batch corresponds to a
+ * range of offsets for a given Kafka topic/partition
+ */
+ @Experimental
+ def createRDD[
+ K: ClassTag,
+ V: ClassTag,
+ KD <: Decoder[K]: ClassTag,
+ VD <: Decoder[V]: ClassTag](
+ sc: SparkContext,
+ kafkaParams: Map[String, String],
+ offsetRanges: Array[OffsetRange]
+ ): RDD[(K, V)] = {
+ val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
+ val leaders = leadersForRanges(kafkaParams, offsetRanges)
+ new KafkaRDD[K, V, KD, VD, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
+ }
+
+ /**
+ * :: Experimental ::
+ * Create a RDD from Kafka using offset ranges for each topic and partition. This allows you
+ * specify the Kafka leader to connect to (to optimize fetching) and access the message as well
+ * as the metadata.
+ *
+ * @param sc SparkContext object
+ * @param kafkaParams Kafka
+ * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers"
+ * to be set with Kafka broker(s) (NOT zookeeper servers) specified in
+ * host1:port1,host2:port2 form.
+ * @param offsetRanges Each OffsetRange in the batch corresponds to a
+ * range of offsets for a given Kafka topic/partition
+ * @param leaders Kafka brokers for each TopicAndPartition in offsetRanges. May be an empty map,
+ * in which case leaders will be looked up on the driver.
+ * @param messageHandler Function for translating each message and metadata into the desired type
+ */
+ @Experimental
+ def createRDD[
+ K: ClassTag,
+ V: ClassTag,
+ KD <: Decoder[K]: ClassTag,
+ VD <: Decoder[V]: ClassTag,
+ R: ClassTag](
+ sc: SparkContext,
+ kafkaParams: Map[String, String],
+ offsetRanges: Array[OffsetRange],
+ leaders: Map[TopicAndPartition, Broker],
+ messageHandler: MessageAndMetadata[K, V] => R
+ ): RDD[R] = {
+ val leaderMap = if (leaders.isEmpty) {
+ leadersForRanges(kafkaParams, offsetRanges)
+ } else {
+ // This could be avoided by refactoring KafkaRDD.leaders and KafkaCluster to use Broker
+ leaders.map {
+ case (tp: TopicAndPartition, Broker(host, port)) => (tp, (host, port))
+ }.toMap
+ }
+ new KafkaRDD[K, V, KD, VD, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler)
+ }
+
+
+ /**
+ * Create a RDD from Kafka using offset ranges for each topic and partition.
+ *
+ * @param jsc JavaSparkContext object
+ * @param kafkaParams Kafka
+ * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers"
+ * to be set with Kafka broker(s) (NOT zookeeper servers) specified in
+ * host1:port1,host2:port2 form.
+ * @param offsetRanges Each OffsetRange in the batch corresponds to a
+ * range of offsets for a given Kafka topic/partition
+ */
+ @Experimental
+ def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V]](
+ jsc: JavaSparkContext,
+ keyClass: Class[K],
+ valueClass: Class[V],
+ keyDecoderClass: Class[KD],
+ valueDecoderClass: Class[VD],
+ kafkaParams: JMap[String, String],
+ offsetRanges: Array[OffsetRange]
+ ): JavaPairRDD[K, V] = {
+ implicit val keyCmt: ClassTag[K] = ClassTag(keyClass)
+ implicit val valueCmt: ClassTag[V] = ClassTag(valueClass)
+ implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass)
+ implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass)
+ new JavaPairRDD(createRDD[K, V, KD, VD](
+ jsc.sc, Map(kafkaParams.toSeq: _*), offsetRanges))
+ }
+
+ /**
+ * :: Experimental ::
+ * Create a RDD from Kafka using offset ranges for each topic and partition. This allows you
+ * specify the Kafka leader to connect to (to optimize fetching) and access the message as well
+ * as the metadata.
+ *
+ * @param jsc JavaSparkContext object
+ * @param kafkaParams Kafka
+ * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers"
+ * to be set with Kafka broker(s) (NOT zookeeper servers) specified in
+ * host1:port1,host2:port2 form.
+ * @param offsetRanges Each OffsetRange in the batch corresponds to a
+ * range of offsets for a given Kafka topic/partition
+ * @param leaders Kafka brokers for each TopicAndPartition in offsetRanges. May be an empty map,
+ * in which case leaders will be looked up on the driver.
+ * @param messageHandler Function for translating each message and metadata into the desired type
+ */
+ @Experimental
+ def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V], R](
+ jsc: JavaSparkContext,
+ keyClass: Class[K],
+ valueClass: Class[V],
+ keyDecoderClass: Class[KD],
+ valueDecoderClass: Class[VD],
+ recordClass: Class[R],
+ kafkaParams: JMap[String, String],
+ offsetRanges: Array[OffsetRange],
+ leaders: JMap[TopicAndPartition, Broker],
+ messageHandler: JFunction[MessageAndMetadata[K, V], R]
+ ): JavaRDD[R] = {
+ implicit val keyCmt: ClassTag[K] = ClassTag(keyClass)
+ implicit val valueCmt: ClassTag[V] = ClassTag(valueClass)
+ implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass)
+ implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass)
+ implicit val recordCmt: ClassTag[R] = ClassTag(recordClass)
+ val leaderMap = Map(leaders.toSeq: _*)
+ createRDD[K, V, KD, VD, R](
+ jsc.sc, Map(kafkaParams.toSeq: _*), offsetRanges, leaderMap, messageHandler.call _)
+ }
+
+ /**
+ * :: Experimental ::
+ * Create an input stream that directly pulls messages from Kafka Brokers
+ * without using any receiver. This stream can guarantee that each message
+ * from Kafka is included in transformations exactly once (see points below).
+ *
+ * Points to note:
+ * - No receivers: This stream does not use any receiver. It directly queries Kafka
+ * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked
+ * by the stream itself. For interoperability with Kafka monitoring tools that depend on
+ * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application.
+ * You can access the offsets used in each batch from the generated RDDs (see
+ * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]).
+ * - Failure Recovery: To recover from driver failures, you have to enable checkpointing
+ * in the [[StreamingContext]]. The information on consumed offset can be
+ * recovered from the checkpoint. See the programming guide for details (constraints, etc.).
+ * - End-to-end semantics: This stream ensures that every records is effectively received and
+ * transformed exactly once, but gives no guarantees on whether the transformed data are
+ * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure
+ * that the output operation is idempotent, or use transactions to output records atomically.
+ * See the programming guide for more details.
+ *
+ * @param ssc StreamingContext object
+ * @param kafkaParams Kafka
+ * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers"
+ * to be set with Kafka broker(s) (NOT zookeeper servers) specified in
+ * host1:port1,host2:port2 form.
+ * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive)
+ * starting point of the stream
+ * @param messageHandler Function for translating each message and metadata into the desired type
+ */
+ @Experimental
+ def createDirectStream[
+ K: ClassTag,
+ V: ClassTag,
+ KD <: Decoder[K]: ClassTag,
+ VD <: Decoder[V]: ClassTag,
+ R: ClassTag] (
+ ssc: StreamingContext,
+ kafkaParams: Map[String, String],
+ fromOffsets: Map[TopicAndPartition, Long],
+ messageHandler: MessageAndMetadata[K, V] => R
+ ): InputDStream[R] = {
+ new DirectKafkaInputDStream[K, V, KD, VD, R](
+ ssc, kafkaParams, fromOffsets, messageHandler)
+ }
+
+ /**
+ * :: Experimental ::
+ * Create an input stream that directly pulls messages from Kafka Brokers
+ * without using any receiver. This stream can guarantee that each message
+ * from Kafka is included in transformations exactly once (see points below).
+ *
+ * Points to note:
+ * - No receivers: This stream does not use any receiver. It directly queries Kafka
+ * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked
+ * by the stream itself. For interoperability with Kafka monitoring tools that depend on
+ * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application.
+ * You can access the offsets used in each batch from the generated RDDs (see
+ * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]).
+ * - Failure Recovery: To recover from driver failures, you have to enable checkpointing
+ * in the [[StreamingContext]]. The information on consumed offset can be
+ * recovered from the checkpoint. See the programming guide for details (constraints, etc.).
+ * - End-to-end semantics: This stream ensures that every records is effectively received and
+ * transformed exactly once, but gives no guarantees on whether the transformed data are
+ * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure
+ * that the output operation is idempotent, or use transactions to output records atomically.
+ * See the programming guide for more details.
+ *
+ * @param ssc StreamingContext object
+ * @param kafkaParams Kafka
+ * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers"
+ * to be set with Kafka broker(s) (NOT zookeeper servers), specified in
+ * host1:port1,host2:port2 form.
+ * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest"
+ * to determine where the stream starts (defaults to "largest")
+ * @param topics Names of the topics to consume
+ */
+ @Experimental
+ def createDirectStream[
+ K: ClassTag,
+ V: ClassTag,
+ KD <: Decoder[K]: ClassTag,
+ VD <: Decoder[V]: ClassTag] (
+ ssc: StreamingContext,
+ kafkaParams: Map[String, String],
+ topics: Set[String]
+ ): InputDStream[(K, V)] = {
+ val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
+ val kc = new KafkaCluster(kafkaParams)
+ val reset = kafkaParams.get("auto.offset.reset").map(_.toLowerCase)
+
+ (for {
+ topicPartitions <- kc.getPartitions(topics).right
+ leaderOffsets <- (if (reset == Some("smallest")) {
+ kc.getEarliestLeaderOffsets(topicPartitions)
+ } else {
+ kc.getLatestLeaderOffsets(topicPartitions)
+ }).right
+ } yield {
+ val fromOffsets = leaderOffsets.map { case (tp, lo) =>
+ (tp, lo.offset)
+ }
+ new DirectKafkaInputDStream[K, V, KD, VD, (K, V)](
+ ssc, kafkaParams, fromOffsets, messageHandler)
+ }).fold(
+ errs => throw new SparkException(errs.mkString("\n")),
+ ok => ok
+ )
+ }
+
+ /**
+ * :: Experimental ::
+ * Create an input stream that directly pulls messages from Kafka Brokers
+ * without using any receiver. This stream can guarantee that each message
+ * from Kafka is included in transformations exactly once (see points below).
+ *
+ * Points to note:
+ * - No receivers: This stream does not use any receiver. It directly queries Kafka
+ * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked
+ * by the stream itself. For interoperability with Kafka monitoring tools that depend on
+ * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application.
+ * You can access the offsets used in each batch from the generated RDDs (see
+ * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]).
+ * - Failure Recovery: To recover from driver failures, you have to enable checkpointing
+ * in the [[StreamingContext]]. The information on consumed offset can be
+ * recovered from the checkpoint. See the programming guide for details (constraints, etc.).
+ * - End-to-end semantics: This stream ensures that every records is effectively received and
+ * transformed exactly once, but gives no guarantees on whether the transformed data are
+ * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure
+ * that the output operation is idempotent, or use transactions to output records atomically.
+ * See the programming guide for more details.
+ *
+ * @param jssc JavaStreamingContext object
+ * @param keyClass Class of the keys in the Kafka records
+ * @param valueClass Class of the values in the Kafka records
+ * @param keyDecoderClass Class of the key decoder
+ * @param valueDecoderClass Class of the value decoder
+ * @param recordClass Class of the records in DStream
+ * @param kafkaParams Kafka
+ * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers"
+ * to be set with Kafka broker(s) (NOT zookeeper servers), specified in
+ * host1:port1,host2:port2 form.
+ * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive)
+ * starting point of the stream
+ * @param messageHandler Function for translating each message and metadata into the desired type
+ */
+ @Experimental
+ def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V], R](
+ jssc: JavaStreamingContext,
+ keyClass: Class[K],
+ valueClass: Class[V],
+ keyDecoderClass: Class[KD],
+ valueDecoderClass: Class[VD],
+ recordClass: Class[R],
+ kafkaParams: JMap[String, String],
+ fromOffsets: JMap[TopicAndPartition, JLong],
+ messageHandler: JFunction[MessageAndMetadata[K, V], R]
+ ): JavaInputDStream[R] = {
+ implicit val keyCmt: ClassTag[K] = ClassTag(keyClass)
+ implicit val valueCmt: ClassTag[V] = ClassTag(valueClass)
+ implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass)
+ implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass)
+ implicit val recordCmt: ClassTag[R] = ClassTag(recordClass)
+ createDirectStream[K, V, KD, VD, R](
+ jssc.ssc,
+ Map(kafkaParams.toSeq: _*),
+ Map(fromOffsets.mapValues { _.longValue() }.toSeq: _*),
+ messageHandler.call _
+ )
+ }
+
+ /**
+ * :: Experimental ::
+ * Create an input stream that directly pulls messages from Kafka Brokers
+ * without using any receiver. This stream can guarantee that each message
+ * from Kafka is included in transformations exactly once (see points below).
+ *
+ * Points to note:
+ * - No receivers: This stream does not use any receiver. It directly queries Kafka
+ * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked
+ * by the stream itself. For interoperability with Kafka monitoring tools that depend on
+ * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application.
+ * You can access the offsets used in each batch from the generated RDDs (see
+ * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]).
+ * - Failure Recovery: To recover from driver failures, you have to enable checkpointing
+ * in the [[StreamingContext]]. The information on consumed offset can be
+ * recovered from the checkpoint. See the programming guide for details (constraints, etc.).
+ * - End-to-end semantics: This stream ensures that every records is effectively received and
+ * transformed exactly once, but gives no guarantees on whether the transformed data are
+ * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure
+ * that the output operation is idempotent, or use transactions to output records atomically.
+ * See the programming guide for more details.
+ *
+ * @param jssc JavaStreamingContext object
+ * @param keyClass Class of the keys in the Kafka records
+ * @param valueClass Class of the values in the Kafka records
+ * @param keyDecoderClass Class of the key decoder
+ * @param valueDecoderClass Class type of the value decoder
+ * @param kafkaParams Kafka
+ * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers"
+ * to be set with Kafka broker(s) (NOT zookeeper servers), specified in
+ * host1:port1,host2:port2 form.
+ * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest"
+ * to determine where the stream starts (defaults to "largest")
+ * @param topics Names of the topics to consume
+ */
+ @Experimental
+ def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V]](
+ jssc: JavaStreamingContext,
+ keyClass: Class[K],
+ valueClass: Class[V],
+ keyDecoderClass: Class[KD],
+ valueDecoderClass: Class[VD],
+ kafkaParams: JMap[String, String],
+ topics: JSet[String]
+ ): JavaPairInputDStream[K, V] = {
+ implicit val keyCmt: ClassTag[K] = ClassTag(keyClass)
+ implicit val valueCmt: ClassTag[V] = ClassTag(valueClass)
+ implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass)
+ implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass)
+ createDirectStream[K, V, KD, VD](
+ jssc.ssc,
+ Map(kafkaParams.toSeq: _*),
+ Set(topics.toSeq: _*)
+ )
+ }
+}
+
+/**
+ * This is a helper class that wraps the KafkaUtils.createStream() into more
+ * Python-friendly class and function so that it can be easily
+ * instantiated and called from Python's KafkaUtils (see SPARK-6027).
+ *
+ * The zero-arg constructor helps instantiate this class from the Class object
+ * classOf[KafkaUtilsPythonHelper].newInstance(), and the createStream()
+ * takes care of known parameters instead of passing them from Python
+ */
+private class KafkaUtilsPythonHelper {
+ def createStream(
+ jssc: JavaStreamingContext,
+ kafkaParams: JMap[String, String],
+ topics: JMap[String, JInt],
+ storageLevel: StorageLevel): JavaPairReceiverInputDStream[Array[Byte], Array[Byte]] = {
+ KafkaUtils.createStream[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder](
+ jssc,
+ classOf[Array[Byte]],
+ classOf[Array[Byte]],
+ classOf[DefaultDecoder],
+ classOf[DefaultDecoder],
+ kafkaParams,
+ topics,
+ storageLevel)
+ }
}
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala
new file mode 100644
index 0000000000000..9c3dfeb8f5928
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala
@@ -0,0 +1,111 @@
+/*
+ * 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.streaming.kafka
+
+import kafka.common.TopicAndPartition
+
+import org.apache.spark.annotation.Experimental
+
+/**
+ * :: Experimental ::
+ * Represents any object that has a collection of [[OffsetRange]]s. This can be used access the
+ * offset ranges in RDDs generated by the direct Kafka DStream (see
+ * [[KafkaUtils.createDirectStream()]]).
+ * {{{
+ * KafkaUtils.createDirectStream(...).foreachRDD { rdd =>
+ * val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges
+ * ...
+ * }
+ * }}}
+ */
+@Experimental
+trait HasOffsetRanges {
+ def offsetRanges: Array[OffsetRange]
+}
+
+/**
+ * :: Experimental ::
+ * Represents a range of offsets from a single Kafka TopicAndPartition. Instances of this class
+ * can be created with `OffsetRange.create()`.
+ */
+@Experimental
+final class OffsetRange private(
+ /** Kafka topic name */
+ val topic: String,
+ /** Kafka partition id */
+ val partition: Int,
+ /** inclusive starting offset */
+ val fromOffset: Long,
+ /** exclusive ending offset */
+ val untilOffset: Long) extends Serializable {
+ import OffsetRange.OffsetRangeTuple
+
+ override def equals(obj: Any): Boolean = obj match {
+ case that: OffsetRange =>
+ this.topic == that.topic &&
+ this.partition == that.partition &&
+ this.fromOffset == that.fromOffset &&
+ this.untilOffset == that.untilOffset
+ case _ => false
+ }
+
+ override def hashCode(): Int = {
+ toTuple.hashCode()
+ }
+
+ override def toString(): String = {
+ s"OffsetRange(topic: '$topic', partition: $partition, range: [$fromOffset -> $untilOffset]"
+ }
+
+ /** this is to avoid ClassNotFoundException during checkpoint restore */
+ private[streaming]
+ def toTuple: OffsetRangeTuple = (topic, partition, fromOffset, untilOffset)
+}
+
+/**
+ * :: Experimental ::
+ * Companion object the provides methods to create instances of [[OffsetRange]].
+ */
+@Experimental
+object OffsetRange {
+ def create(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange =
+ new OffsetRange(topic, partition, fromOffset, untilOffset)
+
+ def create(
+ topicAndPartition: TopicAndPartition,
+ fromOffset: Long,
+ untilOffset: Long): OffsetRange =
+ new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset)
+
+ def apply(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange =
+ new OffsetRange(topic, partition, fromOffset, untilOffset)
+
+ def apply(
+ topicAndPartition: TopicAndPartition,
+ fromOffset: Long,
+ untilOffset: Long): OffsetRange =
+ new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset)
+
+ /** this is to avoid ClassNotFoundException during checkpoint restore */
+ private[kafka]
+ type OffsetRangeTuple = (String, Int, Long, Long)
+
+ private[kafka]
+ def apply(t: OffsetRangeTuple) =
+ new OffsetRange(t._1, t._2, t._3, t._4)
+}
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala
index be734b80272d1..c4a44c1822c39 100644
--- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala
@@ -201,12 +201,31 @@ class ReliableKafkaReceiver[
topicPartitionOffsetMap.clear()
}
- /** Store the ready-to-be-stored block and commit the related offsets to zookeeper. */
+ /**
+ * Store the ready-to-be-stored block and commit the related offsets to zookeeper. This method
+ * will try a fixed number of times to push the block. If the push fails, the receiver is stopped.
+ */
private def storeBlockAndCommitOffset(
blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = {
- store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[(K, V)]])
- Option(blockOffsetMap.get(blockId)).foreach(commitOffset)
- blockOffsetMap.remove(blockId)
+ var count = 0
+ var pushed = false
+ var exception: Exception = null
+ while (!pushed && count <= 3) {
+ try {
+ store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[(K, V)]])
+ pushed = true
+ } catch {
+ case ex: Exception =>
+ count += 1
+ exception = ex
+ }
+ }
+ if (pushed) {
+ Option(blockOffsetMap.get(blockId)).foreach(commitOffset)
+ blockOffsetMap.remove(blockId)
+ } else {
+ stop("Error while storing block into Spark", exception)
+ }
}
/**
diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java
new file mode 100644
index 0000000000000..d6ca6d58b5665
--- /dev/null
+++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java
@@ -0,0 +1,154 @@
+/*
+ * 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.streaming.kafka;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Arrays;
+
+import scala.Tuple2;
+
+import kafka.common.TopicAndPartition;
+import kafka.message.MessageAndMetadata;
+import kafka.serializer.StringDecoder;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.streaming.Durations;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+
+public class JavaDirectKafkaStreamSuite implements Serializable {
+ private transient JavaStreamingContext ssc = null;
+ private transient KafkaStreamSuiteBase suiteBase = null;
+
+ @Before
+ public void setUp() {
+ suiteBase = new KafkaStreamSuiteBase() { };
+ suiteBase.setupKafka();
+ System.clearProperty("spark.driver.port");
+ SparkConf sparkConf = new SparkConf()
+ .setMaster("local[4]").setAppName(this.getClass().getSimpleName());
+ ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200));
+ }
+
+ @After
+ public void tearDown() {
+ ssc.stop();
+ ssc = null;
+ System.clearProperty("spark.driver.port");
+ suiteBase.tearDownKafka();
+ }
+
+ @Test
+ public void testKafkaStream() throws InterruptedException {
+ String topic1 = "topic1";
+ String topic2 = "topic2";
+
+ String[] topic1data = createTopicAndSendData(topic1);
+ String[] topic2data = createTopicAndSendData(topic2);
+
+ HashSet sent = new HashSet();
+ sent.addAll(Arrays.asList(topic1data));
+ sent.addAll(Arrays.asList(topic2data));
+
+ HashMap kafkaParams = new HashMap();
+ kafkaParams.put("metadata.broker.list", suiteBase.brokerAddress());
+ kafkaParams.put("auto.offset.reset", "smallest");
+
+ JavaDStream stream1 = KafkaUtils.createDirectStream(
+ ssc,
+ String.class,
+ String.class,
+ StringDecoder.class,
+ StringDecoder.class,
+ kafkaParams,
+ topicToSet(topic1)
+ ).map(
+ new Function, String>() {
+ @Override
+ public String call(Tuple2 kv) throws Exception {
+ return kv._2();
+ }
+ }
+ );
+
+ JavaDStream stream2 = KafkaUtils.createDirectStream(
+ ssc,
+ String.class,
+ String.class,
+ StringDecoder.class,
+ StringDecoder.class,
+ String.class,
+ kafkaParams,
+ topicOffsetToMap(topic2, (long) 0),
+ new Function, String>() {
+ @Override
+ public String call(MessageAndMetadata msgAndMd) throws Exception {
+ return msgAndMd.message();
+ }
+ }
+ );
+ JavaDStream unifiedStream = stream1.union(stream2);
+
+ final HashSet result = new HashSet();
+ unifiedStream.foreachRDD(
+ new Function, Void>() {
+ @Override
+ public Void call(JavaRDD rdd) throws Exception {
+ result.addAll(rdd.collect());
+ return null;
+ }
+ }
+ );
+ ssc.start();
+ long startTime = System.currentTimeMillis();
+ boolean matches = false;
+ while (!matches && System.currentTimeMillis() - startTime < 20000) {
+ matches = sent.size() == result.size();
+ Thread.sleep(50);
+ }
+ Assert.assertEquals(sent, result);
+ ssc.stop();
+ }
+
+ private HashSet topicToSet(String topic) {
+ HashSet topicSet = new HashSet();
+ topicSet.add(topic);
+ return topicSet;
+ }
+
+ private HashMap topicOffsetToMap(String topic, Long offsetToStart) {
+ HashMap topicMap = new HashMap();
+ topicMap.put(new TopicAndPartition(topic, 0), offsetToStart);
+ return topicMap;
+ }
+
+ private String[] createTopicAndSendData(String topic) {
+ String[] data = { topic + "-1", topic + "-2", topic + "-3"};
+ suiteBase.createTopic(topic);
+ suiteBase.sendMessages(topic, data);
+ return data;
+ }
+}
diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java
new file mode 100644
index 0000000000000..4477b81827c70
--- /dev/null
+++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java
@@ -0,0 +1,151 @@
+/*
+ * 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.streaming.kafka;
+
+import java.io.Serializable;
+import java.util.HashMap;
+
+import scala.Tuple2;
+
+import kafka.common.TopicAndPartition;
+import kafka.message.MessageAndMetadata;
+import kafka.serializer.StringDecoder;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+
+public class JavaKafkaRDDSuite implements Serializable {
+ private transient JavaSparkContext sc = null;
+ private transient KafkaStreamSuiteBase suiteBase = null;
+
+ @Before
+ public void setUp() {
+ suiteBase = new KafkaStreamSuiteBase() { };
+ suiteBase.setupKafka();
+ System.clearProperty("spark.driver.port");
+ SparkConf sparkConf = new SparkConf()
+ .setMaster("local[4]").setAppName(this.getClass().getSimpleName());
+ sc = new JavaSparkContext(sparkConf);
+ }
+
+ @After
+ public void tearDown() {
+ sc.stop();
+ sc = null;
+ System.clearProperty("spark.driver.port");
+ suiteBase.tearDownKafka();
+ }
+
+ @Test
+ public void testKafkaRDD() throws InterruptedException {
+ String topic1 = "topic1";
+ String topic2 = "topic2";
+
+ String[] topic1data = createTopicAndSendData(topic1);
+ String[] topic2data = createTopicAndSendData(topic2);
+
+ HashMap kafkaParams = new HashMap();
+ kafkaParams.put("metadata.broker.list", suiteBase.brokerAddress());
+
+ OffsetRange[] offsetRanges = {
+ OffsetRange.create(topic1, 0, 0, 1),
+ OffsetRange.create(topic2, 0, 0, 1)
+ };
+
+ HashMap emptyLeaders = new HashMap();
+ HashMap leaders = new HashMap();
+ String[] hostAndPort = suiteBase.brokerAddress().split(":");
+ Broker broker = Broker.create(hostAndPort[0], Integer.parseInt(hostAndPort[1]));
+ leaders.put(new TopicAndPartition(topic1, 0), broker);
+ leaders.put(new TopicAndPartition(topic2, 0), broker);
+
+ JavaRDD rdd1 = KafkaUtils.createRDD(
+ sc,
+ String.class,
+ String.class,
+ StringDecoder.class,
+ StringDecoder.class,
+ kafkaParams,
+ offsetRanges
+ ).map(
+ new Function, String>() {
+ @Override
+ public String call(Tuple2 kv) throws Exception {
+ return kv._2();
+ }
+ }
+ );
+
+ JavaRDD rdd2 = KafkaUtils.createRDD(
+ sc,
+ String.class,
+ String.class,
+ StringDecoder.class,
+ StringDecoder.class,
+ String.class,
+ kafkaParams,
+ offsetRanges,
+ emptyLeaders,
+ new Function, String>() {
+ @Override
+ public String call(MessageAndMetadata msgAndMd) throws Exception {
+ return msgAndMd.message();
+ }
+ }
+ );
+
+ JavaRDD rdd3 = KafkaUtils.createRDD(
+ sc,
+ String.class,
+ String.class,
+ StringDecoder.class,
+ StringDecoder.class,
+ String.class,
+ kafkaParams,
+ offsetRanges,
+ leaders,
+ new Function, String>() {
+ @Override
+ public String call(MessageAndMetadata msgAndMd) throws Exception {
+ return msgAndMd.message();
+ }
+ }
+ );
+
+ // just making sure the java user apis work; the scala tests handle logic corner cases
+ long count1 = rdd1.count();
+ long count2 = rdd2.count();
+ long count3 = rdd3.count();
+ Assert.assertTrue(count1 > 0);
+ Assert.assertEquals(count1, count2);
+ Assert.assertEquals(count1, count3);
+ }
+
+ private String[] createTopicAndSendData(String topic) {
+ String[] data = { topic + "-1", topic + "-2", topic + "-3"};
+ suiteBase.createTopic(topic);
+ suiteBase.sendMessages(topic, data);
+ return data;
+ }
+}
diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
index 6e1abf3f385ee..bad0a93eb2e84 100644
--- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
+++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
@@ -22,27 +22,25 @@
import java.util.List;
import java.util.Random;
-import org.apache.spark.SparkConf;
-import org.apache.spark.streaming.Duration;
import scala.Predef;
import scala.Tuple2;
import scala.collection.JavaConverters;
-import junit.framework.Assert;
-
import kafka.serializer.StringDecoder;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.Duration;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
import org.apache.spark.streaming.api.java.JavaStreamingContext;
-import org.junit.Test;
-import org.junit.After;
-import org.junit.Before;
-
public class JavaKafkaStreamSuite implements Serializable {
private transient JavaStreamingContext ssc = null;
private transient Random random = new Random();
@@ -79,9 +77,10 @@ public void testKafkaStream() throws InterruptedException {
suiteBase.createTopic(topic);
HashMap tmp = new HashMap(sent);
- suiteBase.produceAndSendMessage(topic,
+ suiteBase.sendMessages(topic,
JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap(
- Predef.>conforms()));
+ Predef.>conforms())
+ );
HashMap kafkaParams = new HashMap();
kafkaParams.put("zookeeper.connect", suiteBase.zkAddress());
diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala
new file mode 100644
index 0000000000000..17ca9d145d665
--- /dev/null
+++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala
@@ -0,0 +1,306 @@
+/*
+ * 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.streaming.kafka
+
+import java.io.File
+
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import kafka.common.TopicAndPartition
+import kafka.message.MessageAndMetadata
+import kafka.serializer.StringDecoder
+import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll}
+import org.scalatest.concurrent.Eventually
+
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time}
+import org.apache.spark.streaming.dstream.DStream
+import org.apache.spark.util.Utils
+
+class DirectKafkaStreamSuite extends KafkaStreamSuiteBase
+ with BeforeAndAfter with BeforeAndAfterAll with Eventually {
+ val sparkConf = new SparkConf()
+ .setMaster("local[4]")
+ .setAppName(this.getClass.getSimpleName)
+
+ var sc: SparkContext = _
+ var ssc: StreamingContext = _
+ var testDir: File = _
+
+ override def beforeAll {
+ setupKafka()
+ }
+
+ override def afterAll {
+ tearDownKafka()
+ }
+
+ after {
+ if (ssc != null) {
+ ssc.stop()
+ sc = null
+ }
+ if (sc != null) {
+ sc.stop()
+ }
+ if (testDir != null) {
+ Utils.deleteRecursively(testDir)
+ }
+ }
+
+
+ test("basic stream receiving with multiple topics and smallest starting offset") {
+ val topics = Set("basic1", "basic2", "basic3")
+ val data = Map("a" -> 7, "b" -> 9)
+ topics.foreach { t =>
+ createTopic(t)
+ sendMessages(t, data)
+ }
+ val totalSent = data.values.sum * topics.size
+ val kafkaParams = Map(
+ "metadata.broker.list" -> s"$brokerAddress",
+ "auto.offset.reset" -> "smallest"
+ )
+
+ ssc = new StreamingContext(sparkConf, Milliseconds(200))
+ val stream = withClue("Error creating direct stream") {
+ KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder](
+ ssc, kafkaParams, topics)
+ }
+
+ val allReceived = new ArrayBuffer[(String, String)]
+
+ stream.foreachRDD { rdd =>
+ // Get the offset ranges in the RDD
+ val offsets = rdd.asInstanceOf[HasOffsetRanges].offsetRanges
+ val collected = rdd.mapPartitionsWithIndex { (i, iter) =>
+ // For each partition, get size of the range in the partition,
+ // and the number of items in the partition
+ val off = offsets(i)
+ val all = iter.toSeq
+ val partSize = all.size
+ val rangeSize = off.untilOffset - off.fromOffset
+ Iterator((partSize, rangeSize))
+ }.collect
+
+ // Verify whether number of elements in each partition
+ // matches with the corresponding offset range
+ collected.foreach { case (partSize, rangeSize) =>
+ assert(partSize === rangeSize, "offset ranges are wrong")
+ }
+ }
+ stream.foreachRDD { rdd => allReceived ++= rdd.collect() }
+ ssc.start()
+ eventually(timeout(20000.milliseconds), interval(200.milliseconds)) {
+ assert(allReceived.size === totalSent,
+ "didn't get expected number of messages, messages:\n" + allReceived.mkString("\n"))
+ }
+ ssc.stop()
+ }
+
+ test("receiving from largest starting offset") {
+ val topic = "largest"
+ val topicPartition = TopicAndPartition(topic, 0)
+ val data = Map("a" -> 10)
+ createTopic(topic)
+ val kafkaParams = Map(
+ "metadata.broker.list" -> s"$brokerAddress",
+ "auto.offset.reset" -> "largest"
+ )
+ val kc = new KafkaCluster(kafkaParams)
+ def getLatestOffset(): Long = {
+ kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset
+ }
+
+ // Send some initial messages before starting context
+ sendMessages(topic, data)
+ eventually(timeout(10 seconds), interval(20 milliseconds)) {
+ assert(getLatestOffset() > 3)
+ }
+ val offsetBeforeStart = getLatestOffset()
+
+ // Setup context and kafka stream with largest offset
+ ssc = new StreamingContext(sparkConf, Milliseconds(200))
+ val stream = withClue("Error creating direct stream") {
+ KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder](
+ ssc, kafkaParams, Set(topic))
+ }
+ assert(
+ stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]]
+ .fromOffsets(topicPartition) >= offsetBeforeStart,
+ "Start offset not from latest"
+ )
+
+ val collectedData = new mutable.ArrayBuffer[String]()
+ stream.map { _._2 }.foreachRDD { rdd => collectedData ++= rdd.collect() }
+ ssc.start()
+ val newData = Map("b" -> 10)
+ sendMessages(topic, newData)
+ eventually(timeout(10 seconds), interval(50 milliseconds)) {
+ collectedData.contains("b")
+ }
+ assert(!collectedData.contains("a"))
+ }
+
+
+ test("creating stream by offset") {
+ val topic = "offset"
+ val topicPartition = TopicAndPartition(topic, 0)
+ val data = Map("a" -> 10)
+ createTopic(topic)
+ val kafkaParams = Map(
+ "metadata.broker.list" -> s"$brokerAddress",
+ "auto.offset.reset" -> "largest"
+ )
+ val kc = new KafkaCluster(kafkaParams)
+ def getLatestOffset(): Long = {
+ kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset
+ }
+
+ // Send some initial messages before starting context
+ sendMessages(topic, data)
+ eventually(timeout(10 seconds), interval(20 milliseconds)) {
+ assert(getLatestOffset() >= 10)
+ }
+ val offsetBeforeStart = getLatestOffset()
+
+ // Setup context and kafka stream with largest offset
+ ssc = new StreamingContext(sparkConf, Milliseconds(200))
+ val stream = withClue("Error creating direct stream") {
+ KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder, String](
+ ssc, kafkaParams, Map(topicPartition -> 11L),
+ (m: MessageAndMetadata[String, String]) => m.message())
+ }
+ assert(
+ stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]]
+ .fromOffsets(topicPartition) >= offsetBeforeStart,
+ "Start offset not from latest"
+ )
+
+ val collectedData = new mutable.ArrayBuffer[String]()
+ stream.foreachRDD { rdd => collectedData ++= rdd.collect() }
+ ssc.start()
+ val newData = Map("b" -> 10)
+ sendMessages(topic, newData)
+ eventually(timeout(10 seconds), interval(50 milliseconds)) {
+ collectedData.contains("b")
+ }
+ assert(!collectedData.contains("a"))
+ }
+
+ // Test to verify the offset ranges can be recovered from the checkpoints
+ test("offset recovery") {
+ val topic = "recovery"
+ createTopic(topic)
+ testDir = Utils.createTempDir()
+
+ val kafkaParams = Map(
+ "metadata.broker.list" -> s"$brokerAddress",
+ "auto.offset.reset" -> "smallest"
+ )
+
+ // Send data to Kafka and wait for it to be received
+ def sendDataAndWaitForReceive(data: Seq[Int]) {
+ val strings = data.map { _.toString}
+ sendMessages(topic, strings.map { _ -> 1}.toMap)
+ eventually(timeout(10 seconds), interval(50 milliseconds)) {
+ assert(strings.forall { DirectKafkaStreamSuite.collectedData.contains })
+ }
+ }
+
+ // Setup the streaming context
+ ssc = new StreamingContext(sparkConf, Milliseconds(100))
+ val kafkaStream = withClue("Error creating direct stream") {
+ KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder](
+ ssc, kafkaParams, Set(topic))
+ }
+ val keyedStream = kafkaStream.map { v => "key" -> v._2.toInt }
+ val stateStream = keyedStream.updateStateByKey { (values: Seq[Int], state: Option[Int]) =>
+ Some(values.sum + state.getOrElse(0))
+ }
+ ssc.checkpoint(testDir.getAbsolutePath)
+
+ // This is to collect the raw data received from Kafka
+ kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) =>
+ val data = rdd.map { _._2 }.collect()
+ DirectKafkaStreamSuite.collectedData.appendAll(data)
+ }
+
+ // This is ensure all the data is eventually receiving only once
+ stateStream.foreachRDD { (rdd: RDD[(String, Int)]) =>
+ rdd.collect().headOption.foreach { x => DirectKafkaStreamSuite.total = x._2 }
+ }
+ ssc.start()
+
+ // Send some data and wait for them to be received
+ for (i <- (1 to 10).grouped(4)) {
+ sendDataAndWaitForReceive(i)
+ }
+
+ // Verify that offset ranges were generated
+ val offsetRangesBeforeStop = getOffsetRanges(kafkaStream)
+ assert(offsetRangesBeforeStop.size >= 1, "No offset ranges generated")
+ assert(
+ offsetRangesBeforeStop.head._2.forall { _.fromOffset === 0 },
+ "starting offset not zero"
+ )
+ ssc.stop()
+ logInfo("====== RESTARTING ========")
+
+ // Recover context from checkpoints
+ ssc = new StreamingContext(testDir.getAbsolutePath)
+ val recoveredStream = ssc.graph.getInputStreams().head.asInstanceOf[DStream[(String, String)]]
+
+ // Verify offset ranges have been recovered
+ val recoveredOffsetRanges = getOffsetRanges(recoveredStream)
+ assert(recoveredOffsetRanges.size > 0, "No offset ranges recovered")
+ val earlierOffsetRangesAsSets = offsetRangesBeforeStop.map { x => (x._1, x._2.toSet) }
+ assert(
+ recoveredOffsetRanges.forall { or =>
+ earlierOffsetRangesAsSets.contains((or._1, or._2.toSet))
+ },
+ "Recovered ranges are not the same as the ones generated"
+ )
+
+ // Restart context, give more data and verify the total at the end
+ // If the total is write that means each records has been received only once
+ ssc.start()
+ sendDataAndWaitForReceive(11 to 20)
+ eventually(timeout(10 seconds), interval(50 milliseconds)) {
+ assert(DirectKafkaStreamSuite.total === (1 to 20).sum)
+ }
+ ssc.stop()
+ }
+
+ /** Get the generated offset ranges from the DirectKafkaStream */
+ private def getOffsetRanges[K, V](
+ kafkaStream: DStream[(K, V)]): Seq[(Time, Array[OffsetRange])] = {
+ kafkaStream.generatedRDDs.mapValues { rdd =>
+ rdd.asInstanceOf[KafkaRDD[K, V, _, _, (K, V)]].offsetRanges
+ }.toSeq.sortBy { _._1 }
+ }
+}
+
+object DirectKafkaStreamSuite {
+ val collectedData = new mutable.ArrayBuffer[String]()
+ var total = -1L
+}
diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala
new file mode 100644
index 0000000000000..fc9275b7207be
--- /dev/null
+++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala
@@ -0,0 +1,69 @@
+/*
+ * 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.streaming.kafka
+
+import scala.util.Random
+
+import kafka.common.TopicAndPartition
+import org.scalatest.BeforeAndAfterAll
+
+class KafkaClusterSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll {
+ val topic = "kcsuitetopic" + Random.nextInt(10000)
+ val topicAndPartition = TopicAndPartition(topic, 0)
+ var kc: KafkaCluster = null
+
+ override def beforeAll() {
+ setupKafka()
+ createTopic(topic)
+ sendMessages(topic, Map("a" -> 1))
+ kc = new KafkaCluster(Map("metadata.broker.list" -> s"$brokerAddress"))
+ }
+
+ override def afterAll() {
+ tearDownKafka()
+ }
+
+ test("metadata apis") {
+ val leader = kc.findLeaders(Set(topicAndPartition)).right.get(topicAndPartition)
+ val leaderAddress = s"${leader._1}:${leader._2}"
+ assert(leaderAddress === brokerAddress, "didn't get leader")
+
+ val parts = kc.getPartitions(Set(topic)).right.get
+ assert(parts(topicAndPartition), "didn't get partitions")
+ }
+
+ test("leader offset apis") {
+ val earliest = kc.getEarliestLeaderOffsets(Set(topicAndPartition)).right.get
+ assert(earliest(topicAndPartition).offset === 0, "didn't get earliest")
+
+ val latest = kc.getLatestLeaderOffsets(Set(topicAndPartition)).right.get
+ assert(latest(topicAndPartition).offset === 1, "didn't get latest")
+ }
+
+ test("consumer offset apis") {
+ val group = "kcsuitegroup" + Random.nextInt(10000)
+
+ val offset = Random.nextInt(10000)
+
+ val set = kc.setConsumerOffsets(group, Map(topicAndPartition -> offset))
+ assert(set.isRight, "didn't set consumer offsets")
+
+ val get = kc.getConsumerOffsets(group, Set(topicAndPartition)).right.get
+ assert(get(topicAndPartition) === offset, "didn't get consumer offsets")
+ }
+}
diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala
new file mode 100644
index 0000000000000..a223da70b043f
--- /dev/null
+++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala
@@ -0,0 +1,137 @@
+/*
+ * 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.streaming.kafka
+
+import scala.util.Random
+
+import kafka.serializer.StringDecoder
+import kafka.common.TopicAndPartition
+import kafka.message.MessageAndMetadata
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark._
+import org.apache.spark.SparkContext._
+
+class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll {
+ val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName)
+ var sc: SparkContext = _
+ override def beforeAll {
+ sc = new SparkContext(sparkConf)
+
+ setupKafka()
+ }
+
+ override def afterAll {
+ if (sc != null) {
+ sc.stop
+ sc = null
+ }
+ tearDownKafka()
+ }
+
+ test("basic usage") {
+ val topic = "topicbasic"
+ createTopic(topic)
+ val messages = Set("the", "quick", "brown", "fox")
+ sendMessages(topic, messages.toArray)
+
+
+ val kafkaParams = Map("metadata.broker.list" -> brokerAddress,
+ "group.id" -> s"test-consumer-${Random.nextInt(10000)}")
+
+ val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size))
+
+ val rdd = KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder](
+ sc, kafkaParams, offsetRanges)
+
+ val received = rdd.map(_._2).collect.toSet
+ assert(received === messages)
+ }
+
+ test("iterator boundary conditions") {
+ // the idea is to find e.g. off-by-one errors between what kafka has available and the rdd
+ val topic = "topic1"
+ val sent = Map("a" -> 5, "b" -> 3, "c" -> 10)
+ createTopic(topic)
+
+ val kafkaParams = Map("metadata.broker.list" -> brokerAddress,
+ "group.id" -> s"test-consumer-${Random.nextInt(10000)}")
+
+ val kc = new KafkaCluster(kafkaParams)
+
+ // this is the "lots of messages" case
+ sendMessages(topic, sent)
+ // rdd defined from leaders after sending messages, should get the number sent
+ val rdd = getRdd(kc, Set(topic))
+
+ assert(rdd.isDefined)
+ assert(rdd.get.count === sent.values.sum, "didn't get all sent messages")
+
+ val ranges = rdd.get.asInstanceOf[HasOffsetRanges]
+ .offsetRanges.map(o => TopicAndPartition(o.topic, o.partition) -> o.untilOffset).toMap
+
+ kc.setConsumerOffsets(kafkaParams("group.id"), ranges)
+
+ // this is the "0 messages" case
+ val rdd2 = getRdd(kc, Set(topic))
+ // shouldn't get anything, since message is sent after rdd was defined
+ val sentOnlyOne = Map("d" -> 1)
+
+ sendMessages(topic, sentOnlyOne)
+ assert(rdd2.isDefined)
+ assert(rdd2.get.count === 0, "got messages when there shouldn't be any")
+
+ // this is the "exactly 1 message" case, namely the single message from sentOnlyOne above
+ val rdd3 = getRdd(kc, Set(topic))
+ // send lots of messages after rdd was defined, they shouldn't show up
+ sendMessages(topic, Map("extra" -> 22))
+
+ assert(rdd3.isDefined)
+ assert(rdd3.get.count === sentOnlyOne.values.sum, "didn't get exactly one message")
+
+ }
+
+ // get an rdd from the committed consumer offsets until the latest leader offsets,
+ private def getRdd(kc: KafkaCluster, topics: Set[String]) = {
+ val groupId = kc.kafkaParams("group.id")
+ def consumerOffsets(topicPartitions: Set[TopicAndPartition]) = {
+ kc.getConsumerOffsets(groupId, topicPartitions).right.toOption.orElse(
+ kc.getEarliestLeaderOffsets(topicPartitions).right.toOption.map { offs =>
+ offs.map(kv => kv._1 -> kv._2.offset)
+ }
+ )
+ }
+ kc.getPartitions(topics).right.toOption.flatMap { topicPartitions =>
+ consumerOffsets(topicPartitions).flatMap { from =>
+ kc.getLatestLeaderOffsets(topicPartitions).right.toOption.map { until =>
+ val offsetRanges = from.map { case (tp: TopicAndPartition, fromOffset: Long) =>
+ OffsetRange(tp.topic, tp.partition, fromOffset, until(tp).offset)
+ }.toArray
+
+ val leaders = until.map { case (tp: TopicAndPartition, lo: KafkaCluster.LeaderOffset) =>
+ tp -> Broker(lo.host, lo.port)
+ }.toMap
+
+ KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder, String](
+ sc, kc.kafkaParams, offsetRanges, leaders,
+ (mmd: MessageAndMetadata[String, String]) => s"${mmd.offset} ${mmd.message}")
+ }
+ }
+ }
+ }
+}
diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
index b19c053ebfc44..e4966eebb9b34 100644
--- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
+++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
@@ -26,7 +26,7 @@ import scala.concurrent.duration._
import scala.language.postfixOps
import scala.util.Random
-import kafka.admin.CreateTopicCommand
+import kafka.admin.AdminUtils
import kafka.common.{KafkaException, TopicAndPartition}
import kafka.producer.{KeyedMessage, Producer, ProducerConfig}
import kafka.serializer.{StringDecoder, StringEncoder}
@@ -48,30 +48,41 @@ import org.apache.spark.util.Utils
*/
abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Logging {
- var zkAddress: String = _
- var zkClient: ZkClient = _
-
private val zkHost = "localhost"
+ private var zkPort: Int = 0
private val zkConnectionTimeout = 6000
private val zkSessionTimeout = 6000
private var zookeeper: EmbeddedZookeeper = _
- private var zkPort: Int = 0
+ private val brokerHost = "localhost"
private var brokerPort = 9092
private var brokerConf: KafkaConfig = _
private var server: KafkaServer = _
private var producer: Producer[String, String] = _
+ private var zkReady = false
+ private var brokerReady = false
+
+ protected var zkClient: ZkClient = _
+
+ def zkAddress: String = {
+ assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address")
+ s"$zkHost:$zkPort"
+ }
+
+ def brokerAddress: String = {
+ assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address")
+ s"$brokerHost:$brokerPort"
+ }
def setupKafka() {
// Zookeeper server startup
zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort")
// Get the actual zookeeper binding port
zkPort = zookeeper.actualPort
- zkAddress = s"$zkHost:$zkPort"
- logInfo("==================== 0 ====================")
+ zkReady = true
+ logInfo("==================== Zookeeper Started ====================")
- zkClient = new ZkClient(zkAddress, zkSessionTimeout, zkConnectionTimeout,
- ZKStringSerializer)
- logInfo("==================== 1 ====================")
+ zkClient = new ZkClient(zkAddress, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer)
+ logInfo("==================== Zookeeper Client Created ====================")
// Kafka broker startup
var bindSuccess: Boolean = false
@@ -80,9 +91,8 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Loggin
val brokerProps = getBrokerConfig()
brokerConf = new KafkaConfig(brokerProps)
server = new KafkaServer(brokerConf)
- logInfo("==================== 2 ====================")
server.startup()
- logInfo("==================== 3 ====================")
+ logInfo("==================== Kafka Broker Started ====================")
bindSuccess = true
} catch {
case e: KafkaException =>
@@ -94,10 +104,13 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Loggin
}
Thread.sleep(2000)
- logInfo("==================== 4 ====================")
+ logInfo("==================== Kafka + Zookeeper Ready ====================")
+ brokerReady = true
}
def tearDownKafka() {
+ brokerReady = false
+ zkReady = false
if (producer != null) {
producer.close()
producer = null
@@ -121,26 +134,23 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Loggin
}
}
- private def createTestMessage(topic: String, sent: Map[String, Int])
- : Seq[KeyedMessage[String, String]] = {
- val messages = for ((s, freq) <- sent; i <- 0 until freq) yield {
- new KeyedMessage[String, String](topic, s)
- }
- messages.toSeq
- }
-
def createTopic(topic: String) {
- CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0")
- logInfo("==================== 5 ====================")
+ AdminUtils.createTopic(zkClient, topic, 1, 1)
// wait until metadata is propagated
waitUntilMetadataIsPropagated(topic, 0)
+ logInfo(s"==================== Topic $topic Created ====================")
}
- def produceAndSendMessage(topic: String, sent: Map[String, Int]) {
+ def sendMessages(topic: String, messageToFreq: Map[String, Int]) {
+ val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray
+ sendMessages(topic, messages)
+ }
+
+ def sendMessages(topic: String, messages: Array[String]) {
producer = new Producer[String, String](new ProducerConfig(getProducerConfig()))
- producer.send(createTestMessage(topic, sent): _*)
+ producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*)
producer.close()
- logInfo("==================== 6 ====================")
+ logInfo(s"==================== Sent Messages: ${messages.mkString(", ")} ====================")
}
private def getBrokerConfig(): Properties = {
@@ -164,9 +174,9 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Loggin
}
private def waitUntilMetadataIsPropagated(topic: String, partition: Int) {
- eventually(timeout(1000 milliseconds), interval(100 milliseconds)) {
+ eventually(timeout(10000 milliseconds), interval(100 milliseconds)) {
assert(
- server.apis.leaderCache.keySet.contains(TopicAndPartition(topic, partition)),
+ server.apis.metadataCache.containsTopicAndPartition(topic, partition),
s"Partition [$topic, $partition] metadata not propagated after timeout"
)
}
@@ -218,7 +228,7 @@ class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter {
val topic = "topic1"
val sent = Map("a" -> 5, "b" -> 3, "c" -> 10)
createTopic(topic)
- produceAndSendMessage(topic, sent)
+ sendMessages(topic, sent)
val kafkaParams = Map("zookeeper.connect" -> zkAddress,
"group.id" -> s"test-consumer-${Random.nextInt(10000)}",
diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala
index 64ccc92c81fa9..fc53c23abda85 100644
--- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala
+++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala
@@ -79,7 +79,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter
test("Reliable Kafka input stream with single topic") {
var topic = "test-topic"
createTopic(topic)
- produceAndSendMessage(topic, data)
+ sendMessages(topic, data)
// Verify whether the offset of this group/topic/partition is 0 before starting.
assert(getCommitOffset(groupId, topic, 0) === None)
@@ -111,7 +111,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter
val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1)
topics.foreach { case (t, _) =>
createTopic(t)
- produceAndSendMessage(t, data)
+ sendMessages(t, data)
}
// Before started, verify all the group/topic/partition offsets are 0.
diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml
index 560c8b9d18276..a344f000c5002 100644
--- a/external/mqtt/pom.xml
+++ b/external/mqtt/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala
index 77661f71ada21..3c0ef94cb0fab 100644
--- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala
+++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala
@@ -17,23 +17,23 @@
package org.apache.spark.streaming.mqtt
+import java.io.IOException
+import java.util.concurrent.Executors
+import java.util.Properties
+
+import scala.collection.JavaConversions._
import scala.collection.Map
import scala.collection.mutable.HashMap
-import scala.collection.JavaConversions._
import scala.reflect.ClassTag
-import java.util.Properties
-import java.util.concurrent.Executors
-import java.io.IOException
-
+import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken
import org.eclipse.paho.client.mqttv3.MqttCallback
import org.eclipse.paho.client.mqttv3.MqttClient
import org.eclipse.paho.client.mqttv3.MqttClientPersistence
-import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence
-import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken
import org.eclipse.paho.client.mqttv3.MqttException
import org.eclipse.paho.client.mqttv3.MqttMessage
import org.eclipse.paho.client.mqttv3.MqttTopic
+import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence
import org.apache.spark.Logging
import org.apache.spark.storage.StorageLevel
@@ -55,14 +55,14 @@ class MQTTInputDStream(
brokerUrl: String,
topic: String,
storageLevel: StorageLevel
- ) extends ReceiverInputDStream[String](ssc_) with Logging {
-
+ ) extends ReceiverInputDStream[String](ssc_) {
+
def getReceiver(): Receiver[String] = {
new MQTTReceiver(brokerUrl, topic, storageLevel)
}
}
-private[streaming]
+private[streaming]
class MQTTReceiver(
brokerUrl: String,
topic: String,
@@ -72,38 +72,40 @@ class MQTTReceiver(
def onStop() {
}
-
+
def onStart() {
- // Set up persistence for messages
+ // Set up persistence for messages
val persistence = new MemoryPersistence()
// Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance
val client = new MqttClient(brokerUrl, MqttClient.generateClientId(), persistence)
- // Connect to MqttBroker
- client.connect()
-
- // Subscribe to Mqtt topic
- client.subscribe(topic)
-
// Callback automatically triggers as and when new message arrives on specified topic
- val callback: MqttCallback = new MqttCallback() {
+ val callback = new MqttCallback() {
// Handles Mqtt message
- override def messageArrived(arg0: String, arg1: MqttMessage) {
- store(new String(arg1.getPayload(),"utf-8"))
+ override def messageArrived(topic: String, message: MqttMessage) {
+ store(new String(message.getPayload(),"utf-8"))
}
- override def deliveryComplete(arg0: IMqttDeliveryToken) {
+ override def deliveryComplete(token: IMqttDeliveryToken) {
}
- override def connectionLost(arg0: Throwable) {
- restart("Connection lost ", arg0)
+ override def connectionLost(cause: Throwable) {
+ restart("Connection lost ", cause)
}
}
- // Set up callback for MqttClient
+ // Set up callback for MqttClient. This needs to happen before
+ // connecting or subscribing, otherwise messages may be lost
client.setCallback(callback)
+
+ // Connect to MqttBroker
+ client.connect()
+
+ // Subscribe to Mqtt topic
+ client.subscribe(topic)
+
}
}
diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala
index c5ffe51f9986c..1142d0f56ba34 100644
--- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala
+++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala
@@ -17,10 +17,11 @@
package org.apache.spark.streaming.mqtt
+import scala.reflect.ClassTag
+
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.StreamingContext
import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext, JavaDStream}
-import scala.reflect.ClassTag
import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream}
object MQTTUtils {
diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
index 1e24da7f5f60c..cfedb5a042a35 100644
--- a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
+++ b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
@@ -31,7 +31,7 @@ public void setUp() {
SparkConf conf = new SparkConf()
.setMaster("local[2]")
.setAppName("test")
- .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
+ .set("spark.streaming.clock", "org.apache.spark.util.ManualClock");
ssc = new JavaStreamingContext(conf, new Duration(1000));
ssc.checkpoint("checkpoint");
}
diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
index fe53a29cba0c9..0f3298af6234a 100644
--- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
+++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
@@ -18,6 +18,8 @@
package org.apache.spark.streaming.mqtt
import java.net.{URI, ServerSocket}
+import java.util.concurrent.CountDownLatch
+import java.util.concurrent.TimeUnit
import scala.concurrent.duration._
import scala.language.postfixOps
@@ -32,14 +34,16 @@ import org.scalatest.concurrent.Eventually
import org.apache.spark.streaming.{Milliseconds, StreamingContext}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.dstream.ReceiverInputDStream
+import org.apache.spark.streaming.scheduler.StreamingListener
+import org.apache.spark.streaming.scheduler.StreamingListenerReceiverStarted
import org.apache.spark.SparkConf
import org.apache.spark.util.Utils
class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter {
private val batchDuration = Milliseconds(500)
- private val master: String = "local[2]"
- private val framework: String = this.getClass.getSimpleName
+ private val master = "local[2]"
+ private val framework = this.getClass.getSimpleName
private val freePort = findFreePort()
private val brokerUri = "//localhost:" + freePort
private val topic = "def"
@@ -65,9 +69,9 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter {
test("mqtt input stream") {
val sendMessage = "MQTT demo for spark streaming"
- val receiveStream: ReceiverInputDStream[String] =
+ val receiveStream =
MQTTUtils.createStream(ssc, "tcp:" + brokerUri, topic, StorageLevel.MEMORY_ONLY)
- var receiveMessage: List[String] = List()
+ @volatile var receiveMessage: List[String] = List()
receiveStream.foreachRDD { rdd =>
if (rdd.collect.length > 0) {
receiveMessage = receiveMessage ::: List(rdd.first)
@@ -75,6 +79,11 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter {
}
}
ssc.start()
+
+ // wait for the receiver to start before publishing data, or we risk failing
+ // the test nondeterministically. See SPARK-4631
+ waitForReceiverToStart()
+
publishData(sendMessage)
eventually(timeout(10000 milliseconds), interval(100 milliseconds)) {
assert(sendMessage.equals(receiveMessage(0)))
@@ -84,6 +93,7 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter {
private def setupMQTT() {
broker = new BrokerService()
+ broker.setDataDirectoryFile(Utils.createTempDir())
connector = new TransportConnector()
connector.setName("mqtt")
connector.setUri(new URI("mqtt:" + brokerUri))
@@ -113,16 +123,22 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter {
def publishData(data: String): Unit = {
var client: MqttClient = null
try {
- val persistence: MqttClientPersistence = new MqttDefaultFilePersistence(persistenceDir.getAbsolutePath)
+ val persistence = new MqttDefaultFilePersistence(persistenceDir.getAbsolutePath)
client = new MqttClient("tcp:" + brokerUri, MqttClient.generateClientId(), persistence)
client.connect()
if (client.isConnected) {
- val msgTopic: MqttTopic = client.getTopic(topic)
- val message: MqttMessage = new MqttMessage(data.getBytes("utf-8"))
+ val msgTopic = client.getTopic(topic)
+ val message = new MqttMessage(data.getBytes("utf-8"))
message.setQos(1)
message.setRetained(true)
- for (i <- 0 to 100) {
- msgTopic.publish(message)
+
+ for (i <- 0 to 10) {
+ try {
+ msgTopic.publish(message)
+ } catch {
+ case e: MqttException if e.getReasonCode == MqttException.REASON_CODE_MAX_INFLIGHT =>
+ Thread.sleep(50) // wait for Spark streaming to consume something from the message queue
+ }
}
}
} finally {
@@ -131,4 +147,18 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter {
client = null
}
}
+
+ /**
+ * Block until at least one receiver has started or timeout occurs.
+ */
+ private def waitForReceiverToStart() = {
+ val latch = new CountDownLatch(1)
+ ssc.addStreamingListener(new StreamingListener {
+ override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) {
+ latch.countDown()
+ }
+ })
+
+ assert(latch.await(10, TimeUnit.SECONDS), "Timeout waiting for receiver to start.")
+ }
}
diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml
index da6ffe7662f63..e95853f005ce2 100644
--- a/external/twitter/pom.xml
+++ b/external/twitter/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
index 1e24da7f5f60c..cfedb5a042a35 100644
--- a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
+++ b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
@@ -31,7 +31,7 @@ public void setUp() {
SparkConf conf = new SparkConf()
.setMaster("local[2]")
.setAppName("test")
- .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
+ .set("spark.streaming.clock", "org.apache.spark.util.ManualClock");
ssc = new JavaStreamingContext(conf, new Duration(1000));
ssc.checkpoint("checkpoint");
}
diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml
index e919c2c9b19ea..9b3475d7c3dc2 100644
--- a/external/zeromq/pom.xml
+++ b/external/zeromq/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
index 1e24da7f5f60c..cfedb5a042a35 100644
--- a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
+++ b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
@@ -31,7 +31,7 @@ public void setUp() {
SparkConf conf = new SparkConf()
.setMaster("local[2]")
.setAppName("test")
- .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
+ .set("spark.streaming.clock", "org.apache.spark.util.ManualClock");
ssc = new JavaStreamingContext(conf, new Duration(1000));
ssc.checkpoint("checkpoint");
}
diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml
index 0fb431808bacd..bc2f8be10c9ce 100644
--- a/extras/java8-tests/pom.xml
+++ b/extras/java8-tests/pom.xml
@@ -19,7 +19,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml
index c815eda52bda7..7e49a71907336 100644
--- a/extras/kinesis-asl/pom.xml
+++ b/extras/kinesis-asl/pom.xml
@@ -19,7 +19,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
@@ -67,11 +67,6 @@
scalacheck_${scala.binary.version}
test
-
- org.easymock
- easymockclassextension
- test
-
com.novocode
junit-interface
diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala
index 0b80b611cdce7..588e86a1887ec 100644
--- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala
+++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala
@@ -18,9 +18,7 @@ package org.apache.spark.streaming.kinesis
import org.apache.spark.Logging
import org.apache.spark.streaming.Duration
-import org.apache.spark.streaming.util.Clock
-import org.apache.spark.streaming.util.ManualClock
-import org.apache.spark.streaming.util.SystemClock
+import org.apache.spark.util.{Clock, ManualClock, SystemClock}
/**
* This is a helper class for managing checkpoint clocks.
@@ -35,7 +33,7 @@ private[kinesis] class KinesisCheckpointState(
/* Initialize the checkpoint clock using the given currentClock + checkpointInterval millis */
val checkpointClock = new ManualClock()
- checkpointClock.setTime(currentClock.currentTime() + checkpointInterval.milliseconds)
+ checkpointClock.setTime(currentClock.getTimeMillis() + checkpointInterval.milliseconds)
/**
* Check if it's time to checkpoint based on the current time and the derived time
@@ -44,13 +42,13 @@ private[kinesis] class KinesisCheckpointState(
* @return true if it's time to checkpoint
*/
def shouldCheckpoint(): Boolean = {
- new SystemClock().currentTime() > checkpointClock.currentTime()
+ new SystemClock().getTimeMillis() > checkpointClock.getTimeMillis()
}
/**
* Advance the checkpoint clock by the checkpoint interval.
*/
def advanceCheckpoint() = {
- checkpointClock.addToTime(checkpointInterval.milliseconds)
+ checkpointClock.advance(checkpointInterval.milliseconds)
}
}
diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala
index 8ecc2d90160b1..af8cd875b4541 100644
--- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala
+++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala
@@ -104,7 +104,7 @@ private[kinesis] class KinesisRecordProcessor(
logDebug(s"Checkpoint: WorkerId $workerId completed checkpoint of ${batch.size}" +
s" records for shardId $shardId")
logDebug(s"Checkpoint: Next checkpoint is at " +
- s" ${checkpointState.checkpointClock.currentTime()} for shardId $shardId")
+ s" ${checkpointState.checkpointClock.getTimeMillis()} for shardId $shardId")
}
} catch {
case e: Throwable => {
diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala
index 41dbd64c2b1fa..255fe65819608 100644
--- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala
+++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala
@@ -20,17 +20,17 @@ import java.nio.ByteBuffer
import scala.collection.JavaConversions.seqAsJavaList
-import org.apache.spark.annotation.Experimental
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.Milliseconds
import org.apache.spark.streaming.Seconds
import org.apache.spark.streaming.StreamingContext
import org.apache.spark.streaming.TestSuiteBase
-import org.apache.spark.streaming.util.Clock
-import org.apache.spark.streaming.util.ManualClock
+import org.apache.spark.util.{ManualClock, Clock}
+
+import org.mockito.Mockito._
import org.scalatest.BeforeAndAfter
import org.scalatest.Matchers
-import org.scalatest.mock.EasyMockSugar
+import org.scalatest.mock.MockitoSugar
import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException
import com.amazonaws.services.kinesis.clientlibrary.exceptions.KinesisClientLibDependencyException
@@ -42,10 +42,10 @@ import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason
import com.amazonaws.services.kinesis.model.Record
/**
- * Suite of Kinesis streaming receiver tests focusing mostly on the KinesisRecordProcessor
+ * Suite of Kinesis streaming receiver tests focusing mostly on the KinesisRecordProcessor
*/
class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAfter
- with EasyMockSugar {
+ with MockitoSugar {
val app = "TestKinesisReceiver"
val stream = "mySparkStream"
@@ -73,6 +73,14 @@ class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAft
currentClockMock = mock[Clock]
}
+ override def afterFunction(): Unit = {
+ super.afterFunction()
+ // Since this suite was originally written using EasyMock, add this to preserve the old
+ // mocking semantics (see SPARK-5735 for more details)
+ verifyNoMoreInteractions(receiverMock, checkpointerMock, checkpointClockMock,
+ checkpointStateMock, currentClockMock)
+ }
+
test("kinesis utils api") {
val ssc = new StreamingContext(master, framework, batchDuration)
// Tests the API, does not actually test data receiving
@@ -83,193 +91,175 @@ class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAft
}
test("process records including store and checkpoint") {
- val expectedCheckpointIntervalMillis = 10
- expecting {
- receiverMock.isStopped().andReturn(false).once()
- receiverMock.store(record1.getData().array()).once()
- receiverMock.store(record2.getData().array()).once()
- checkpointStateMock.shouldCheckpoint().andReturn(true).once()
- checkpointerMock.checkpoint().once()
- checkpointStateMock.advanceCheckpoint().once()
- }
- whenExecuting(receiverMock, checkpointerMock, checkpointStateMock) {
- val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId,
- checkpointStateMock)
- recordProcessor.processRecords(batch, checkpointerMock)
- }
+ when(receiverMock.isStopped()).thenReturn(false)
+ when(checkpointStateMock.shouldCheckpoint()).thenReturn(true)
+
+ val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock)
+ recordProcessor.processRecords(batch, checkpointerMock)
+
+ verify(receiverMock, times(1)).isStopped()
+ verify(receiverMock, times(1)).store(record1.getData().array())
+ verify(receiverMock, times(1)).store(record2.getData().array())
+ verify(checkpointStateMock, times(1)).shouldCheckpoint()
+ verify(checkpointerMock, times(1)).checkpoint()
+ verify(checkpointStateMock, times(1)).advanceCheckpoint()
}
test("shouldn't store and checkpoint when receiver is stopped") {
- expecting {
- receiverMock.isStopped().andReturn(true).once()
- }
- whenExecuting(receiverMock, checkpointerMock, checkpointStateMock) {
- val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId,
- checkpointStateMock)
- recordProcessor.processRecords(batch, checkpointerMock)
- }
+ when(receiverMock.isStopped()).thenReturn(true)
+
+ val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock)
+ recordProcessor.processRecords(batch, checkpointerMock)
+
+ verify(receiverMock, times(1)).isStopped()
}
test("shouldn't checkpoint when exception occurs during store") {
- expecting {
- receiverMock.isStopped().andReturn(false).once()
- receiverMock.store(record1.getData().array()).andThrow(new RuntimeException()).once()
- }
- whenExecuting(receiverMock, checkpointerMock, checkpointStateMock) {
- intercept[RuntimeException] {
- val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId,
- checkpointStateMock)
- recordProcessor.processRecords(batch, checkpointerMock)
- }
+ when(receiverMock.isStopped()).thenReturn(false)
+ when(receiverMock.store(record1.getData().array())).thenThrow(new RuntimeException())
+
+ intercept[RuntimeException] {
+ val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock)
+ recordProcessor.processRecords(batch, checkpointerMock)
}
+
+ verify(receiverMock, times(1)).isStopped()
+ verify(receiverMock, times(1)).store(record1.getData().array())
}
test("should set checkpoint time to currentTime + checkpoint interval upon instantiation") {
- expecting {
- currentClockMock.currentTime().andReturn(0).once()
- }
- whenExecuting(currentClockMock) {
+ when(currentClockMock.getTimeMillis()).thenReturn(0)
+
val checkpointIntervalMillis = 10
- val checkpointState = new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock)
- assert(checkpointState.checkpointClock.currentTime() == checkpointIntervalMillis)
- }
+ val checkpointState =
+ new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock)
+ assert(checkpointState.checkpointClock.getTimeMillis() == checkpointIntervalMillis)
+
+ verify(currentClockMock, times(1)).getTimeMillis()
}
test("should checkpoint if we have exceeded the checkpoint interval") {
- expecting {
- currentClockMock.currentTime().andReturn(0).once()
- }
- whenExecuting(currentClockMock) {
- val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MinValue), currentClockMock)
- assert(checkpointState.shouldCheckpoint())
- }
+ when(currentClockMock.getTimeMillis()).thenReturn(0)
+
+ val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MinValue), currentClockMock)
+ assert(checkpointState.shouldCheckpoint())
+
+ verify(currentClockMock, times(1)).getTimeMillis()
}
test("shouldn't checkpoint if we have not exceeded the checkpoint interval") {
- expecting {
- currentClockMock.currentTime().andReturn(0).once()
- }
- whenExecuting(currentClockMock) {
- val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MaxValue), currentClockMock)
- assert(!checkpointState.shouldCheckpoint())
- }
+ when(currentClockMock.getTimeMillis()).thenReturn(0)
+
+ val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MaxValue), currentClockMock)
+ assert(!checkpointState.shouldCheckpoint())
+
+ verify(currentClockMock, times(1)).getTimeMillis()
}
test("should add to time when advancing checkpoint") {
- expecting {
- currentClockMock.currentTime().andReturn(0).once()
- }
- whenExecuting(currentClockMock) {
- val checkpointIntervalMillis = 10
- val checkpointState = new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock)
- assert(checkpointState.checkpointClock.currentTime() == checkpointIntervalMillis)
- checkpointState.advanceCheckpoint()
- assert(checkpointState.checkpointClock.currentTime() == (2 * checkpointIntervalMillis))
- }
+ when(currentClockMock.getTimeMillis()).thenReturn(0)
+
+ val checkpointIntervalMillis = 10
+ val checkpointState =
+ new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock)
+ assert(checkpointState.checkpointClock.getTimeMillis() == checkpointIntervalMillis)
+ checkpointState.advanceCheckpoint()
+ assert(checkpointState.checkpointClock.getTimeMillis() == (2 * checkpointIntervalMillis))
+
+ verify(currentClockMock, times(1)).getTimeMillis()
}
test("shutdown should checkpoint if the reason is TERMINATE") {
- expecting {
- checkpointerMock.checkpoint().once()
- }
- whenExecuting(checkpointerMock, checkpointStateMock) {
- val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId,
- checkpointStateMock)
- val reason = ShutdownReason.TERMINATE
- recordProcessor.shutdown(checkpointerMock, reason)
- }
+ val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock)
+ val reason = ShutdownReason.TERMINATE
+ recordProcessor.shutdown(checkpointerMock, reason)
+
+ verify(checkpointerMock, times(1)).checkpoint()
}
test("shutdown should not checkpoint if the reason is something other than TERMINATE") {
- expecting {
- }
- whenExecuting(checkpointerMock, checkpointStateMock) {
- val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId,
- checkpointStateMock)
- recordProcessor.shutdown(checkpointerMock, ShutdownReason.ZOMBIE)
- recordProcessor.shutdown(checkpointerMock, null)
- }
+ val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock)
+ recordProcessor.shutdown(checkpointerMock, ShutdownReason.ZOMBIE)
+ recordProcessor.shutdown(checkpointerMock, null)
+
+ verify(checkpointerMock, never()).checkpoint()
}
test("retry success on first attempt") {
val expectedIsStopped = false
- expecting {
- receiverMock.isStopped().andReturn(expectedIsStopped).once()
- }
- whenExecuting(receiverMock) {
- val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100)
- assert(actualVal == expectedIsStopped)
- }
+ when(receiverMock.isStopped()).thenReturn(expectedIsStopped)
+
+ val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100)
+ assert(actualVal == expectedIsStopped)
+
+ verify(receiverMock, times(1)).isStopped()
}
test("retry success on second attempt after a Kinesis throttling exception") {
val expectedIsStopped = false
- expecting {
- receiverMock.isStopped().andThrow(new ThrottlingException("error message"))
- .andReturn(expectedIsStopped).once()
- }
- whenExecuting(receiverMock) {
- val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100)
- assert(actualVal == expectedIsStopped)
- }
+ when(receiverMock.isStopped())
+ .thenThrow(new ThrottlingException("error message"))
+ .thenReturn(expectedIsStopped)
+
+ val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100)
+ assert(actualVal == expectedIsStopped)
+
+ verify(receiverMock, times(2)).isStopped()
}
test("retry success on second attempt after a Kinesis dependency exception") {
val expectedIsStopped = false
- expecting {
- receiverMock.isStopped().andThrow(new KinesisClientLibDependencyException("error message"))
- .andReturn(expectedIsStopped).once()
- }
- whenExecuting(receiverMock) {
- val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100)
- assert(actualVal == expectedIsStopped)
- }
+ when(receiverMock.isStopped())
+ .thenThrow(new KinesisClientLibDependencyException("error message"))
+ .thenReturn(expectedIsStopped)
+
+ val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100)
+ assert(actualVal == expectedIsStopped)
+
+ verify(receiverMock, times(2)).isStopped()
}
test("retry failed after a shutdown exception") {
- expecting {
- checkpointerMock.checkpoint().andThrow(new ShutdownException("error message")).once()
- }
- whenExecuting(checkpointerMock) {
- intercept[ShutdownException] {
- KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100)
- }
+ when(checkpointerMock.checkpoint()).thenThrow(new ShutdownException("error message"))
+
+ intercept[ShutdownException] {
+ KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100)
}
+
+ verify(checkpointerMock, times(1)).checkpoint()
}
test("retry failed after an invalid state exception") {
- expecting {
- checkpointerMock.checkpoint().andThrow(new InvalidStateException("error message")).once()
- }
- whenExecuting(checkpointerMock) {
- intercept[InvalidStateException] {
- KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100)
- }
+ when(checkpointerMock.checkpoint()).thenThrow(new InvalidStateException("error message"))
+
+ intercept[InvalidStateException] {
+ KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100)
}
+
+ verify(checkpointerMock, times(1)).checkpoint()
}
test("retry failed after unexpected exception") {
- expecting {
- checkpointerMock.checkpoint().andThrow(new RuntimeException("error message")).once()
- }
- whenExecuting(checkpointerMock) {
- intercept[RuntimeException] {
- KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100)
- }
+ when(checkpointerMock.checkpoint()).thenThrow(new RuntimeException("error message"))
+
+ intercept[RuntimeException] {
+ KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100)
}
+
+ verify(checkpointerMock, times(1)).checkpoint()
}
test("retry failed after exhausing all retries") {
val expectedErrorMessage = "final try error message"
- expecting {
- checkpointerMock.checkpoint().andThrow(new ThrottlingException("error message"))
- .andThrow(new ThrottlingException(expectedErrorMessage)).once()
- }
- whenExecuting(checkpointerMock) {
- val exception = intercept[RuntimeException] {
- KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100)
- }
- exception.getMessage().shouldBe(expectedErrorMessage)
+ when(checkpointerMock.checkpoint())
+ .thenThrow(new ThrottlingException("error message"))
+ .thenThrow(new ThrottlingException(expectedErrorMessage))
+
+ val exception = intercept[RuntimeException] {
+ KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100)
}
+ exception.getMessage().shouldBe(expectedErrorMessage)
+
+ verify(checkpointerMock, times(2)).checkpoint()
}
}
diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml
index f2f0aa78b0a4b..6eb29af03f833 100644
--- a/extras/spark-ganglia-lgpl/pom.xml
+++ b/extras/spark-ganglia-lgpl/pom.xml
@@ -19,7 +19,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/graphx/pom.xml b/graphx/pom.xml
index 8fac24b6ed86d..c0d534e185d7f 100644
--- a/graphx/pom.xml
+++ b/graphx/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../pom.xml
@@ -45,9 +45,14 @@
guava
- org.jblas
- jblas
- ${jblas.version}
+ com.github.fommil.netlib
+ core
+ ${netlib.java.version}
+
+
+ net.sourceforge.f2j
+ arpack_combined_all
+ 0.1
org.scalacheck
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
index ab56580a3abc8..8494d06b1cdb7 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
@@ -104,6 +104,18 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab
*/
def checkpoint(): Unit
+ /**
+ * Return whether this Graph has been checkpointed or not.
+ * This returns true iff both the vertices RDD and edges RDD have been checkpointed.
+ */
+ def isCheckpointed: Boolean
+
+ /**
+ * Gets the name of the files to which this Graph was checkpointed.
+ * (The vertices RDD and edges RDD are checkpointed separately.)
+ */
+ def getCheckpointFiles: Seq[String]
+
/**
* Uncaches both vertices and edges of this graph. This is useful in iterative algorithms that
* build a new graph in each iteration.
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
index 4933aecba1286..21187be7678a6 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
@@ -77,7 +77,7 @@ object GraphLoader extends Logging {
if (!line.isEmpty && line(0) != '#') {
val lineArray = line.split("\\s+")
if (lineArray.length < 2) {
- logWarning("Invalid line: " + line)
+ throw new IllegalArgumentException("Invalid line: " + line)
}
val srcId = lineArray(0).toLong
val dstId = lineArray(1).toLong
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
index 09ae3f9f6c09b..40ecff7107109 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
@@ -122,8 +122,11 @@ abstract class VertexRDD[VD](
def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2]
/**
- * Hides vertices that are the same between `this` and `other`; for vertices that are different,
- * keeps the values from `other`.
+ * For each vertex present in both `this` and `other`, `diff` returns only those vertices with
+ * differing values; for values that are different, keeps the values from `other`. This is
+ * only guaranteed to work if the VertexRDDs share a common ancestor.
+ *
+ * @param other the other VertexRDD with which to diff against.
*/
def diff(other: VertexRDD[VD]): VertexRDD[VD]
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala
index f1550ac2e18ad..43a3aea0f6196 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala
@@ -19,7 +19,7 @@ package org.apache.spark.graphx.impl
import scala.reflect.{classTag, ClassTag}
-import org.apache.spark.{OneToOneDependency, HashPartitioner, TaskContext}
+import org.apache.spark.{OneToOneDependency, HashPartitioner}
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel
@@ -70,10 +70,20 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] (
this
}
+ override def getStorageLevel = partitionsRDD.getStorageLevel
+
override def checkpoint() = {
partitionsRDD.checkpoint()
}
-
+
+ override def isCheckpointed: Boolean = {
+ firstParent[(PartitionID, EdgePartition[ED, VD])].isCheckpointed
+ }
+
+ override def getCheckpointFile: Option[String] = {
+ partitionsRDD.getCheckpointFile
+ }
+
/** The number of edges in the RDD. */
override def count(): Long = {
partitionsRDD.map(_._2.size.toLong).reduce(_ + _)
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
index 3f4a900d5b601..90a74d23a26cc 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
@@ -70,6 +70,17 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
replicatedVertexView.edges.checkpoint()
}
+ override def isCheckpointed: Boolean = {
+ vertices.isCheckpointed && replicatedVertexView.edges.isCheckpointed
+ }
+
+ override def getCheckpointFiles: Seq[String] = {
+ Seq(vertices.getCheckpointFile, replicatedVertexView.edges.getCheckpointFile).flatMap {
+ case Some(path) => Seq(path)
+ case None => Seq()
+ }
+ }
+
override def unpersist(blocking: Boolean = true): Graph[VD, ED] = {
unpersistVertices(blocking)
replicatedVertexView.edges.unpersist(blocking)
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala
index 9732c5b00c6d9..904be213147dc 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala
@@ -71,10 +71,20 @@ class VertexRDDImpl[VD] private[graphx] (
this
}
+ override def getStorageLevel = partitionsRDD.getStorageLevel
+
override def checkpoint() = {
partitionsRDD.checkpoint()
}
-
+
+ override def isCheckpointed: Boolean = {
+ firstParent[ShippableVertexPartition[VD]].isCheckpointed
+ }
+
+ override def getCheckpointFile: Option[String] = {
+ partitionsRDD.getCheckpointFile
+ }
+
/** The number of vertices in the RDD. */
override def count(): Long = {
partitionsRDD.map(_.size).reduce(_ + _)
@@ -94,8 +104,14 @@ class VertexRDDImpl[VD] private[graphx] (
this.mapVertexPartitions(_.map(f))
override def diff(other: VertexRDD[VD]): VertexRDD[VD] = {
+ val otherPartition = other match {
+ case other: VertexRDD[_] if this.partitioner == other.partitioner =>
+ other.partitionsRDD
+ case _ =>
+ VertexRDD(other.partitionBy(this.partitioner.get)).partitionsRDD
+ }
val newPartitionsRDD = partitionsRDD.zipPartitions(
- other.partitionsRDD, preservesPartitioning = true
+ otherPartition, preservesPartitioning = true
) { (thisIter, otherIter) =>
val thisPart = thisIter.next()
val otherPart = otherIter.next()
@@ -123,7 +139,7 @@ class VertexRDDImpl[VD] private[graphx] (
// Test if the other vertex is a VertexRDD to choose the optimal join strategy.
// If the other set is a VertexRDD then we use the much more efficient leftZipJoin
other match {
- case other: VertexRDD[_] =>
+ case other: VertexRDD[_] if this.partitioner == other.partitioner =>
leftZipJoin(other)(f)
case _ =>
this.withPartitionsRDD[VD3](
@@ -152,7 +168,7 @@ class VertexRDDImpl[VD] private[graphx] (
// Test if the other vertex is a VertexRDD to choose the optimal join strategy.
// If the other set is a VertexRDD then we use the much more efficient innerZipJoin
other match {
- case other: VertexRDD[_] =>
+ case other: VertexRDD[_] if this.partitioner == other.partitioner =>
innerZipJoin(other)(f)
case _ =>
this.withPartitionsRDD(
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala
index f58587e10a820..1a7178b82e3af 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala
@@ -18,7 +18,9 @@
package org.apache.spark.graphx.lib
import scala.util.Random
-import org.jblas.DoubleMatrix
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
import org.apache.spark.rdd._
import org.apache.spark.graphx._
@@ -37,12 +39,23 @@ object SVDPlusPlus {
var gamma7: Double)
extends Serializable
+ /**
+ * This method is now replaced by the updated version of `run()` and returns exactly
+ * the same result.
+ */
+ @deprecated("Call run()", "1.4.0")
+ def runSVDPlusPlus(edges: RDD[Edge[Double]], conf: Conf)
+ : (Graph[(Array[Double], Array[Double], Double, Double), Double], Double) =
+ {
+ run(edges, conf)
+ }
+
/**
* Implement SVD++ based on "Factorization Meets the Neighborhood:
* a Multifaceted Collaborative Filtering Model",
* available at [[http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf]].
*
- * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^(-0.5)*sum(y)),
+ * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^^-0.5^^*sum(y)),
* see the details on page 6.
*
* @param edges edges for constructing the graph
@@ -52,16 +65,13 @@ object SVDPlusPlus {
* @return a graph with vertex attributes containing the trained model
*/
def run(edges: RDD[Edge[Double]], conf: Conf)
- : (Graph[(DoubleMatrix, DoubleMatrix, Double, Double), Double], Double) =
+ : (Graph[(Array[Double], Array[Double], Double, Double), Double], Double) =
{
// Generate default vertex attribute
- def defaultF(rank: Int): (DoubleMatrix, DoubleMatrix, Double, Double) = {
- val v1 = new DoubleMatrix(rank)
- val v2 = new DoubleMatrix(rank)
- for (i <- 0 until rank) {
- v1.put(i, Random.nextDouble())
- v2.put(i, Random.nextDouble())
- }
+ def defaultF(rank: Int): (Array[Double], Array[Double], Double, Double) = {
+ // TODO: use a fixed random seed
+ val v1 = Array.fill(rank)(Random.nextDouble())
+ val v2 = Array.fill(rank)(Random.nextDouble())
(v1, v2, 0.0, 0.0)
}
@@ -72,38 +82,47 @@ object SVDPlusPlus {
// construct graph
var g = Graph.fromEdges(edges, defaultF(conf.rank)).cache()
+ materialize(g)
+ edges.unpersist()
// Calculate initial bias and norm
val t0 = g.aggregateMessages[(Long, Double)](
ctx => { ctx.sendToSrc((1L, ctx.attr)); ctx.sendToDst((1L, ctx.attr)) },
(g1, g2) => (g1._1 + g2._1, g1._2 + g2._2))
- g = g.outerJoinVertices(t0) {
- (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double),
+ val gJoinT0 = g.outerJoinVertices(t0) {
+ (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double),
msg: Option[(Long, Double)]) =>
(vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1))
- }
+ }.cache()
+ materialize(gJoinT0)
+ g.unpersist()
+ g = gJoinT0
def sendMsgTrainF(conf: Conf, u: Double)
(ctx: EdgeContext[
- (DoubleMatrix, DoubleMatrix, Double, Double),
+ (Array[Double], Array[Double], Double, Double),
Double,
- (DoubleMatrix, DoubleMatrix, Double)]) {
+ (Array[Double], Array[Double], Double)]) {
val (usr, itm) = (ctx.srcAttr, ctx.dstAttr)
val (p, q) = (usr._1, itm._1)
- var pred = u + usr._3 + itm._3 + q.dot(usr._2)
+ val rank = p.length
+ var pred = u + usr._3 + itm._3 + blas.ddot(rank, q, 1, usr._2, 1)
pred = math.max(pred, conf.minVal)
pred = math.min(pred, conf.maxVal)
val err = ctx.attr - pred
- val updateP = q.mul(err)
- .subColumnVector(p.mul(conf.gamma7))
- .mul(conf.gamma2)
- val updateQ = usr._2.mul(err)
- .subColumnVector(q.mul(conf.gamma7))
- .mul(conf.gamma2)
- val updateY = q.mul(err * usr._4)
- .subColumnVector(itm._2.mul(conf.gamma7))
- .mul(conf.gamma2)
+ // updateP = (err * q - conf.gamma7 * p) * conf.gamma2
+ val updateP = q.clone()
+ blas.dscal(rank, err * conf.gamma2, updateP, 1)
+ blas.daxpy(rank, -conf.gamma7 * conf.gamma2, p, 1, updateP, 1)
+ // updateQ = (err * usr._2 - conf.gamma7 * q) * conf.gamma2
+ val updateQ = usr._2.clone()
+ blas.dscal(rank, err * conf.gamma2, updateQ, 1)
+ blas.daxpy(rank, -conf.gamma7 * conf.gamma2, q, 1, updateQ, 1)
+ // updateY = (err * usr._4 * q - conf.gamma7 * itm._2) * conf.gamma2
+ val updateY = q.clone()
+ blas.dscal(rank, err * usr._4 * conf.gamma2, updateY, 1)
+ blas.daxpy(rank, -conf.gamma7 * conf.gamma2, itm._2, 1, updateY, 1)
ctx.sendToSrc((updateP, updateY, (err - conf.gamma6 * usr._3) * conf.gamma1))
ctx.sendToDst((updateQ, updateY, (err - conf.gamma6 * itm._3) * conf.gamma1))
}
@@ -111,49 +130,89 @@ object SVDPlusPlus {
for (i <- 0 until conf.maxIters) {
// Phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes
g.cache()
- val t1 = g.aggregateMessages[DoubleMatrix](
+ val t1 = g.aggregateMessages[Array[Double]](
ctx => ctx.sendToSrc(ctx.dstAttr._2),
- (g1, g2) => g1.addColumnVector(g2))
- g = g.outerJoinVertices(t1) {
- (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double),
- msg: Option[DoubleMatrix]) =>
- if (msg.isDefined) (vd._1, vd._1
- .addColumnVector(msg.get.mul(vd._4)), vd._3, vd._4) else vd
- }
+ (g1, g2) => {
+ val out = g1.clone()
+ blas.daxpy(out.length, 1.0, g2, 1, out, 1)
+ out
+ })
+ val gJoinT1 = g.outerJoinVertices(t1) {
+ (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double),
+ msg: Option[Array[Double]]) =>
+ if (msg.isDefined) {
+ val out = vd._1.clone()
+ blas.daxpy(out.length, vd._4, msg.get, 1, out, 1)
+ (vd._1, out, vd._3, vd._4)
+ } else {
+ vd
+ }
+ }.cache()
+ materialize(gJoinT1)
+ g.unpersist()
+ g = gJoinT1
// Phase 2, update p for user nodes and q, y for item nodes
g.cache()
val t2 = g.aggregateMessages(
sendMsgTrainF(conf, u),
- (g1: (DoubleMatrix, DoubleMatrix, Double), g2: (DoubleMatrix, DoubleMatrix, Double)) =>
- (g1._1.addColumnVector(g2._1), g1._2.addColumnVector(g2._2), g1._3 + g2._3))
- g = g.outerJoinVertices(t2) {
+ (g1: (Array[Double], Array[Double], Double), g2: (Array[Double], Array[Double], Double)) =>
+ {
+ val out1 = g1._1.clone()
+ blas.daxpy(out1.length, 1.0, g2._1, 1, out1, 1)
+ val out2 = g2._2.clone()
+ blas.daxpy(out2.length, 1.0, g2._2, 1, out2, 1)
+ (out1, out2, g1._3 + g2._3)
+ })
+ val gJoinT2 = g.outerJoinVertices(t2) {
(vid: VertexId,
- vd: (DoubleMatrix, DoubleMatrix, Double, Double),
- msg: Option[(DoubleMatrix, DoubleMatrix, Double)]) =>
- (vd._1.addColumnVector(msg.get._1), vd._2.addColumnVector(msg.get._2),
- vd._3 + msg.get._3, vd._4)
- }
+ vd: (Array[Double], Array[Double], Double, Double),
+ msg: Option[(Array[Double], Array[Double], Double)]) => {
+ val out1 = vd._1.clone()
+ blas.daxpy(out1.length, 1.0, msg.get._1, 1, out1, 1)
+ val out2 = vd._2.clone()
+ blas.daxpy(out2.length, 1.0, msg.get._2, 1, out2, 1)
+ (out1, out2, vd._3 + msg.get._3, vd._4)
+ }
+ }.cache()
+ materialize(gJoinT2)
+ g.unpersist()
+ g = gJoinT2
}
// calculate error on training set
def sendMsgTestF(conf: Conf, u: Double)
- (ctx: EdgeContext[(DoubleMatrix, DoubleMatrix, Double, Double), Double, Double]) {
+ (ctx: EdgeContext[(Array[Double], Array[Double], Double, Double), Double, Double]) {
val (usr, itm) = (ctx.srcAttr, ctx.dstAttr)
val (p, q) = (usr._1, itm._1)
- var pred = u + usr._3 + itm._3 + q.dot(usr._2)
+ var pred = u + usr._3 + itm._3 + blas.ddot(q.length, q, 1, usr._2, 1)
pred = math.max(pred, conf.minVal)
pred = math.min(pred, conf.maxVal)
val err = (ctx.attr - pred) * (ctx.attr - pred)
ctx.sendToDst(err)
}
+
g.cache()
val t3 = g.aggregateMessages[Double](sendMsgTestF(conf, u), _ + _)
- g = g.outerJoinVertices(t3) {
- (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double), msg: Option[Double]) =>
+ val gJoinT3 = g.outerJoinVertices(t3) {
+ (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double), msg: Option[Double]) =>
if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd
- }
+ }.cache()
+ materialize(gJoinT3)
+ g.unpersist()
+ g = gJoinT3
- (g, u)
+ // Convert DoubleMatrix to Array[Double]:
+ val newVertices = g.vertices.mapValues(v => (v._1.toArray, v._2.toArray, v._3, v._4))
+ (Graph(newVertices, g.edges), u)
}
+
+ /**
+ * Forces materialization of a Graph by count()ing its RDDs.
+ */
+ private def materialize(g: Graph[_,_]): Unit = {
+ g.vertices.count()
+ g.edges.count()
+ }
+
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala
index 590f0474957dd..179f2843818e0 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala
@@ -61,8 +61,8 @@ object ShortestPaths {
}
def sendMessage(edge: EdgeTriplet[SPMap, _]): Iterator[(VertexId, SPMap)] = {
- val newAttr = incrementMap(edge.srcAttr)
- if (edge.dstAttr != addMaps(newAttr, edge.dstAttr)) Iterator((edge.dstId, newAttr))
+ val newAttr = incrementMap(edge.dstAttr)
+ if (edge.srcAttr != addMaps(newAttr, edge.srcAttr)) Iterator((edge.srcId, newAttr))
else Iterator.empty
}
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala
new file mode 100644
index 0000000000000..eb1dbe52c2fda
--- /dev/null
+++ b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.graphx
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.storage.StorageLevel
+
+class EdgeRDDSuite extends FunSuite with LocalSparkContext {
+
+ test("cache, getStorageLevel") {
+ // test to see if getStorageLevel returns correct value after caching
+ withSpark { sc =>
+ val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3)))
+ val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]]))
+ assert(edges.getStorageLevel == StorageLevel.NONE)
+ edges.cache()
+ assert(edges.getStorageLevel == StorageLevel.MEMORY_ONLY)
+ }
+ }
+
+}
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
index ed9876b8dc21c..b61d9f0fbe5e4 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
@@ -25,6 +25,7 @@ import org.apache.spark.SparkContext
import org.apache.spark.graphx.Graph._
import org.apache.spark.graphx.PartitionStrategy._
import org.apache.spark.rdd._
+import org.apache.spark.storage.StorageLevel
class GraphSuite extends FunSuite with LocalSparkContext {
@@ -375,6 +376,8 @@ class GraphSuite extends FunSuite with LocalSparkContext {
val ring = (0L to 100L).zip((1L to 99L) :+ 0L).map { case (a, b) => Edge(a, b, 1)}
val rdd = sc.parallelize(ring)
val graph = Graph.fromEdges(rdd, 1.0F)
+ assert(!graph.isCheckpointed)
+ assert(graph.getCheckpointFiles.size === 0)
graph.checkpoint()
graph.edges.map(_.attr).count()
graph.vertices.map(_._2).count()
@@ -383,6 +386,22 @@ class GraphSuite extends FunSuite with LocalSparkContext {
val verticesDependencies = graph.vertices.partitionsRDD.dependencies
assert(edgesDependencies.forall(_.rdd.isInstanceOf[CheckpointRDD[_]]))
assert(verticesDependencies.forall(_.rdd.isInstanceOf[CheckpointRDD[_]]))
+ assert(graph.isCheckpointed)
+ assert(graph.getCheckpointFiles.size === 2)
+ }
+ }
+
+ test("cache, getStorageLevel") {
+ // test to see if getStorageLevel returns correct value
+ withSpark { sc =>
+ val verts = sc.parallelize(List((1: VertexId, "a"), (2: VertexId, "b")), 1)
+ val edges = sc.parallelize(List(Edge(1, 2, 0), Edge(2, 1, 0)), 2)
+ val graph = Graph(verts, edges, "", StorageLevel.MEMORY_ONLY, StorageLevel.MEMORY_ONLY)
+ // Note: Before caching, graph.vertices is cached, but graph.edges is not (but graph.edges'
+ // parent RDD is cached).
+ graph.cache()
+ assert(graph.vertices.getStorageLevel == StorageLevel.MEMORY_ONLY)
+ assert(graph.edges.getStorageLevel == StorageLevel.MEMORY_ONLY)
}
}
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala
index 42d3f21dbae98..97533dd3aa6ce 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala
@@ -17,12 +17,11 @@
package org.apache.spark.graphx
-import org.apache.spark.SparkContext
-import org.apache.spark.graphx.Graph._
-import org.apache.spark.graphx.impl.EdgePartition
-import org.apache.spark.rdd._
import org.scalatest.FunSuite
+import org.apache.spark.{HashPartitioner, SparkContext}
+import org.apache.spark.storage.StorageLevel
+
class VertexRDDSuite extends FunSuite with LocalSparkContext {
def vertices(sc: SparkContext, n: Int) = {
@@ -59,6 +58,16 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext {
}
}
+ test("diff vertices with the non-equal number of partitions") {
+ withSpark { sc =>
+ val vertexA = VertexRDD(sc.parallelize(0 until 24, 3).map(i => (i.toLong, 0)))
+ val vertexB = VertexRDD(sc.parallelize(8 until 16, 2).map(i => (i.toLong, 1)))
+ assert(vertexA.partitions.size != vertexB.partitions.size)
+ val vertexC = vertexA.diff(vertexB)
+ assert(vertexC.map(_._1).collect.toSet === (8 until 16).toSet)
+ }
+ }
+
test("leftJoin") {
withSpark { sc =>
val n = 100
@@ -74,6 +83,19 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext {
}
}
+ test("leftJoin vertices with the non-equal number of partitions") {
+ withSpark { sc =>
+ val vertexA = VertexRDD(sc.parallelize(0 until 100, 2).map(i => (i.toLong, 1)))
+ val vertexB = VertexRDD(
+ vertexA.filter(v => v._1 % 2 == 0).partitionBy(new HashPartitioner(3)))
+ assert(vertexA.partitions.size != vertexB.partitions.size)
+ val vertexC = vertexA.leftJoin(vertexB) { (vid, old, newOpt) =>
+ old - newOpt.getOrElse(0)
+ }
+ assert(vertexC.filter(v => v._2 != 0).map(_._1).collect.toSet == (1 to 99 by 2).toSet)
+ }
+ }
+
test("innerJoin") {
withSpark { sc =>
val n = 100
@@ -88,6 +110,19 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext {
(0 to n by 2).map(x => (x.toLong, 0)).toSet) }
}
+ test("innerJoin vertices with the non-equal number of partitions") {
+ withSpark { sc =>
+ val vertexA = VertexRDD(sc.parallelize(0 until 100, 2).map(i => (i.toLong, 1)))
+ val vertexB = VertexRDD(
+ vertexA.filter(v => v._1 % 2 == 0).partitionBy(new HashPartitioner(3)))
+ assert(vertexA.partitions.size != vertexB.partitions.size)
+ val vertexC = vertexA.innerJoin(vertexB) { (vid, old, newVal) =>
+ old - newVal
+ }
+ assert(vertexC.filter(v => v._2 == 0).map(_._1).collect.toSet == (0 to 98 by 2).toSet)
+ }
+ }
+
test("aggregateUsingIndex") {
withSpark { sc =>
val n = 100
@@ -110,4 +145,16 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext {
}
}
+ test("cache, getStorageLevel") {
+ // test to see if getStorageLevel returns correct value after caching
+ withSpark { sc =>
+ val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3)))
+ val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]]))
+ val rdd = VertexRDD(verts, edges, 0, (a: Int, b: Int) => a + b)
+ assert(rdd.getStorageLevel == StorageLevel.NONE)
+ rdd.cache()
+ assert(rdd.getStorageLevel == StorageLevel.MEMORY_ONLY)
+ }
+ }
+
}
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala
index e01df56e94de9..7bd6b7f3c4ab2 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala
@@ -32,11 +32,11 @@ class SVDPlusPlusSuite extends FunSuite with LocalSparkContext {
Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble)
}
val conf = new SVDPlusPlus.Conf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations
- var (graph, u) = SVDPlusPlus.run(edges, conf)
+ val (graph, _) = SVDPlusPlus.run(edges, conf)
graph.cache()
- val err = graph.vertices.collect().map{ case (vid, vd) =>
+ val err = graph.vertices.map { case (vid, vd) =>
if (vid % 2 == 1) vd._4 else 0.0
- }.reduce(_ + _) / graph.triplets.collect().size
+ }.reduce(_ + _) / graph.numEdges
assert(err <= svdppErr)
}
}
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala
index 265827b3341c2..f2c38e79c452c 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala
@@ -40,7 +40,7 @@ class ShortestPathsSuite extends FunSuite with LocalSparkContext {
val graph = Graph.fromEdgeTuples(edges, 1)
val landmarks = Seq(1, 4).map(_.toLong)
val results = ShortestPaths.run(graph, landmarks).vertices.collect.map {
- case (v, spMap) => (v, spMap.mapValues(_.get))
+ case (v, spMap) => (v, spMap.mapValues(i => i))
}
assert(results.toSet === shortestPaths)
}
diff --git a/launcher/pom.xml b/launcher/pom.xml
new file mode 100644
index 0000000000000..ccbd9d0419a98
--- /dev/null
+++ b/launcher/pom.xml
@@ -0,0 +1,83 @@
+
+
+
+
+ 4.0.0
+
+ org.apache.spark
+ spark-parent_2.10
+ 1.3.0-SNAPSHOT
+ ../pom.xml
+
+
+ org.apache.spark
+ spark-launcher_2.10
+ jar
+ Spark Launcher Project
+ http://spark.apache.org/
+
+ launcher
+
+
+
+
+
+ log4j
+ log4j
+ test
+
+
+ junit
+ junit
+ test
+
+
+ org.mockito
+ mockito-all
+ test
+
+
+ org.scalatest
+ scalatest_${scala.binary.version}
+ test
+
+
+ org.slf4j
+ slf4j-api
+ test
+
+
+ org.slf4j
+ slf4j-log4j12
+ test
+
+
+
+
+ org.apache.hadoop
+ hadoop-client
+ test
+
+
+
+
+ target/scala-${scala.binary.version}/classes
+ target/scala-${scala.binary.version}/test-classes
+
+
diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java
new file mode 100644
index 0000000000000..dc90e9e987234
--- /dev/null
+++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java
@@ -0,0 +1,362 @@
+/*
+ * 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.launcher;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.FileInputStream;
+import java.io.InputStreamReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.jar.JarFile;
+import java.util.regex.Pattern;
+
+import static org.apache.spark.launcher.CommandBuilderUtils.*;
+
+/**
+ * Abstract Spark command builder that defines common functionality.
+ */
+abstract class AbstractCommandBuilder {
+
+ boolean verbose;
+ String appName;
+ String appResource;
+ String deployMode;
+ String javaHome;
+ String mainClass;
+ String master;
+ String propertiesFile;
+ final List appArgs;
+ final List jars;
+ final List files;
+ final List pyFiles;
+ final Map childEnv;
+ final Map conf;
+
+ public AbstractCommandBuilder() {
+ this.appArgs = new ArrayList();
+ this.childEnv = new HashMap();
+ this.conf = new HashMap();
+ this.files = new ArrayList();
+ this.jars = new ArrayList();
+ this.pyFiles = new ArrayList();
+ }
+
+ /**
+ * Builds the command to execute.
+ *
+ * @param env A map containing environment variables for the child process. It may already contain
+ * entries defined by the user (such as SPARK_HOME, or those defined by the
+ * SparkLauncher constructor that takes an environment), and may be modified to
+ * include other variables needed by the process to be executed.
+ */
+ abstract List buildCommand(Map env) throws IOException;
+
+ /**
+ * Builds a list of arguments to run java.
+ *
+ * This method finds the java executable to use and appends JVM-specific options for running a
+ * class with Spark in the classpath. It also loads options from the "java-opts" file in the
+ * configuration directory being used.
+ *
+ * Callers should still add at least the class to run, as well as any arguments to pass to the
+ * class.
+ */
+ List buildJavaCommand(String extraClassPath) throws IOException {
+ List cmd = new ArrayList();
+ if (javaHome == null) {
+ cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java"));
+ } else {
+ cmd.add(join(File.separator, javaHome, "bin", "java"));
+ }
+
+ // Load extra JAVA_OPTS from conf/java-opts, if it exists.
+ File javaOpts = new File(join(File.separator, getConfDir(), "java-opts"));
+ if (javaOpts.isFile()) {
+ BufferedReader br = new BufferedReader(new InputStreamReader(
+ new FileInputStream(javaOpts), "UTF-8"));
+ try {
+ String line;
+ while ((line = br.readLine()) != null) {
+ addOptionString(cmd, line);
+ }
+ } finally {
+ br.close();
+ }
+ }
+
+ cmd.add("-cp");
+ cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath)));
+ return cmd;
+ }
+
+ /**
+ * Adds the default perm gen size option for Spark if the VM requires it and the user hasn't
+ * set it.
+ */
+ void addPermGenSizeOpt(List cmd) {
+ // Don't set MaxPermSize for Java 8 and later.
+ String[] version = System.getProperty("java.version").split("\\.");
+ if (Integer.parseInt(version[0]) > 1 || Integer.parseInt(version[1]) > 7) {
+ return;
+ }
+
+ for (String arg : cmd) {
+ if (arg.startsWith("-XX:MaxPermSize=")) {
+ return;
+ }
+ }
+
+ cmd.add("-XX:MaxPermSize=128m");
+ }
+
+ void addOptionString(List cmd, String options) {
+ if (!isEmpty(options)) {
+ for (String opt : parseOptionString(options)) {
+ cmd.add(opt);
+ }
+ }
+ }
+
+ /**
+ * Builds the classpath for the application. Returns a list with one classpath entry per element;
+ * each entry is formatted in the way expected by java.net.URLClassLoader (more
+ * specifically, with trailing slashes for directories).
+ */
+ List buildClassPath(String appClassPath) throws IOException {
+ String sparkHome = getSparkHome();
+ String scala = getScalaVersion();
+
+ List cp = new ArrayList();
+ addToClassPath(cp, getenv("SPARK_CLASSPATH"));
+ addToClassPath(cp, appClassPath);
+
+ addToClassPath(cp, getConfDir());
+
+ boolean prependClasses = !isEmpty(getenv("SPARK_PREPEND_CLASSES"));
+ boolean isTesting = "1".equals(getenv("SPARK_TESTING"));
+ if (prependClasses || isTesting) {
+ List projects = Arrays.asList("core", "repl", "mllib", "bagel", "graphx",
+ "streaming", "tools", "sql/catalyst", "sql/core", "sql/hive", "sql/hive-thriftserver",
+ "yarn", "launcher");
+ if (prependClasses) {
+ System.err.println(
+ "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of " +
+ "assembly.");
+ for (String project : projects) {
+ addToClassPath(cp, String.format("%s/%s/target/scala-%s/classes", sparkHome, project,
+ scala));
+ }
+ }
+ if (isTesting) {
+ for (String project : projects) {
+ addToClassPath(cp, String.format("%s/%s/target/scala-%s/test-classes", sparkHome,
+ project, scala));
+ }
+ }
+
+ // Add this path to include jars that are shaded in the final deliverable created during
+ // the maven build. These jars are copied to this directory during the build.
+ addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome));
+ }
+
+ String assembly = findAssembly(scala);
+ addToClassPath(cp, assembly);
+
+ // When Hive support is needed, Datanucleus jars must be included on the classpath. Datanucleus
+ // jars do not work if only included in the uber jar as plugin.xml metadata is lost. Both sbt
+ // and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is built
+ // with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark
+ // assembly is built for Hive, before actually populating the CLASSPATH with the jars.
+ //
+ // This block also serves as a check for SPARK-1703, when the assembly jar is built with
+ // Java 7 and ends up with too many files, causing issues with other JDK versions.
+ boolean needsDataNucleus = false;
+ JarFile assemblyJar = null;
+ try {
+ assemblyJar = new JarFile(assembly);
+ needsDataNucleus = assemblyJar.getEntry("org/apache/hadoop/hive/ql/exec/") != null;
+ } catch (IOException ioe) {
+ if (ioe.getMessage().indexOf("invalid CEN header") >= 0) {
+ System.err.println(
+ "Loading Spark jar failed.\n" +
+ "This is likely because Spark was compiled with Java 7 and run\n" +
+ "with Java 6 (see SPARK-1703). Please use Java 7 to run Spark\n" +
+ "or build Spark with Java 6.");
+ System.exit(1);
+ } else {
+ throw ioe;
+ }
+ } finally {
+ if (assemblyJar != null) {
+ try {
+ assemblyJar.close();
+ } catch (IOException e) {
+ // Ignore.
+ }
+ }
+ }
+
+ if (needsDataNucleus) {
+ System.err.println("Spark assembly has been built with Hive, including Datanucleus jars " +
+ "in classpath.");
+ File libdir;
+ if (new File(sparkHome, "RELEASE").isFile()) {
+ libdir = new File(sparkHome, "lib");
+ } else {
+ libdir = new File(sparkHome, "lib_managed/jars");
+ }
+
+ checkState(libdir.isDirectory(), "Library directory '%s' does not exist.",
+ libdir.getAbsolutePath());
+ for (File jar : libdir.listFiles()) {
+ if (jar.getName().startsWith("datanucleus-")) {
+ addToClassPath(cp, jar.getAbsolutePath());
+ }
+ }
+ }
+
+ addToClassPath(cp, getenv("HADOOP_CONF_DIR"));
+ addToClassPath(cp, getenv("YARN_CONF_DIR"));
+ addToClassPath(cp, getenv("SPARK_DIST_CLASSPATH"));
+ return cp;
+ }
+
+ /**
+ * Adds entries to the classpath.
+ *
+ * @param cp List to which the new entries are appended.
+ * @param entries New classpath entries (separated by File.pathSeparator).
+ */
+ private void addToClassPath(List cp, String entries) {
+ if (isEmpty(entries)) {
+ return;
+ }
+ String[] split = entries.split(Pattern.quote(File.pathSeparator));
+ for (String entry : split) {
+ if (!isEmpty(entry)) {
+ if (new File(entry).isDirectory() && !entry.endsWith(File.separator)) {
+ entry += File.separator;
+ }
+ cp.add(entry);
+ }
+ }
+ }
+
+ String getScalaVersion() {
+ String scala = getenv("SPARK_SCALA_VERSION");
+ if (scala != null) {
+ return scala;
+ }
+
+ String sparkHome = getSparkHome();
+ File scala210 = new File(sparkHome, "assembly/target/scala-2.10");
+ File scala211 = new File(sparkHome, "assembly/target/scala-2.11");
+ checkState(!scala210.isDirectory() || !scala211.isDirectory(),
+ "Presence of build for both scala versions (2.10 and 2.11) detected.\n" +
+ "Either clean one of them or set SPARK_SCALA_VERSION in your environment.");
+ if (scala210.isDirectory()) {
+ return "2.10";
+ } else {
+ checkState(scala211.isDirectory(), "Cannot find any assembly build directories.");
+ return "2.11";
+ }
+ }
+
+ String getSparkHome() {
+ String path = getenv(ENV_SPARK_HOME);
+ checkState(path != null,
+ "Spark home not found; set it explicitly or use the SPARK_HOME environment variable.");
+ return path;
+ }
+
+ /**
+ * Loads the configuration file for the application, if it exists. This is either the
+ * user-specified properties file, or the spark-defaults.conf file under the Spark configuration
+ * directory.
+ */
+ Properties loadPropertiesFile() throws IOException {
+ Properties props = new Properties();
+ File propsFile;
+ if (propertiesFile != null) {
+ propsFile = new File(propertiesFile);
+ checkArgument(propsFile.isFile(), "Invalid properties file '%s'.", propertiesFile);
+ } else {
+ propsFile = new File(getConfDir(), DEFAULT_PROPERTIES_FILE);
+ }
+
+ if (propsFile.isFile()) {
+ FileInputStream fd = null;
+ try {
+ fd = new FileInputStream(propsFile);
+ props.load(new InputStreamReader(fd, "UTF-8"));
+ } finally {
+ if (fd != null) {
+ try {
+ fd.close();
+ } catch (IOException e) {
+ // Ignore.
+ }
+ }
+ }
+ }
+
+ return props;
+ }
+
+ String getenv(String key) {
+ return firstNonEmpty(childEnv.get(key), System.getenv(key));
+ }
+
+ private String findAssembly(String scalaVersion) {
+ String sparkHome = getSparkHome();
+ File libdir;
+ if (new File(sparkHome, "RELEASE").isFile()) {
+ libdir = new File(sparkHome, "lib");
+ checkState(libdir.isDirectory(), "Library directory '%s' does not exist.",
+ libdir.getAbsolutePath());
+ } else {
+ libdir = new File(sparkHome, String.format("assembly/target/scala-%s", scalaVersion));
+ }
+
+ final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar");
+ FileFilter filter = new FileFilter() {
+ @Override
+ public boolean accept(File file) {
+ return file.isFile() && re.matcher(file.getName()).matches();
+ }
+ };
+ File[] assemblies = libdir.listFiles(filter);
+ checkState(assemblies != null && assemblies.length > 0, "No assemblies found in '%s'.", libdir);
+ checkState(assemblies.length == 1, "Multiple assemblies found in '%s'.", libdir);
+ return assemblies[0].getAbsolutePath();
+ }
+
+ private String getConfDir() {
+ String confDir = getenv("SPARK_CONF_DIR");
+ return confDir != null ? confDir : join(File.separator, getSparkHome(), "conf");
+ }
+
+}
diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java
new file mode 100644
index 0000000000000..9b04732afee14
--- /dev/null
+++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java
@@ -0,0 +1,296 @@
+/*
+ * 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.launcher;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Helper methods for command builders.
+ */
+class CommandBuilderUtils {
+
+ static final String DEFAULT_MEM = "512m";
+ static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf";
+ static final String ENV_SPARK_HOME = "SPARK_HOME";
+
+ /** Returns whether the given string is null or empty. */
+ static boolean isEmpty(String s) {
+ return s == null || s.isEmpty();
+ }
+
+ /** Joins a list of strings using the given separator. */
+ static String join(String sep, String... elements) {
+ StringBuilder sb = new StringBuilder();
+ for (String e : elements) {
+ if (e != null) {
+ if (sb.length() > 0) {
+ sb.append(sep);
+ }
+ sb.append(e);
+ }
+ }
+ return sb.toString();
+ }
+
+ /** Joins a list of strings using the given separator. */
+ static String join(String sep, Iterable elements) {
+ StringBuilder sb = new StringBuilder();
+ for (String e : elements) {
+ if (e != null) {
+ if (sb.length() > 0) {
+ sb.append(sep);
+ }
+ sb.append(e);
+ }
+ }
+ return sb.toString();
+ }
+
+ /**
+ * Returns the first non-empty value mapped to the given key in the given maps, or null otherwise.
+ */
+ static String firstNonEmptyValue(String key, Map, ?>... maps) {
+ for (Map, ?> map : maps) {
+ String value = (String) map.get(key);
+ if (!isEmpty(value)) {
+ return value;
+ }
+ }
+ return null;
+ }
+
+ /** Returns the first non-empty, non-null string in the given list, or null otherwise. */
+ static String firstNonEmpty(String... candidates) {
+ for (String s : candidates) {
+ if (!isEmpty(s)) {
+ return s;
+ }
+ }
+ return null;
+ }
+
+ /** Returns the name of the env variable that holds the native library path. */
+ static String getLibPathEnvName() {
+ if (isWindows()) {
+ return "PATH";
+ }
+
+ String os = System.getProperty("os.name");
+ if (os.startsWith("Mac OS X")) {
+ return "DYLD_LIBRARY_PATH";
+ } else {
+ return "LD_LIBRARY_PATH";
+ }
+ }
+
+ /** Returns whether the OS is Windows. */
+ static boolean isWindows() {
+ String os = System.getProperty("os.name");
+ return os.startsWith("Windows");
+ }
+
+ /**
+ * Updates the user environment, appending the given pathList to the existing value of the given
+ * environment variable (or setting it if it hasn't yet been set).
+ */
+ static void mergeEnvPathList(Map userEnv, String envKey, String pathList) {
+ if (!isEmpty(pathList)) {
+ String current = firstNonEmpty(userEnv.get(envKey), System.getenv(envKey));
+ userEnv.put(envKey, join(File.pathSeparator, current, pathList));
+ }
+ }
+
+ /**
+ * Parse a string as if it were a list of arguments, following bash semantics.
+ * For example:
+ *
+ * Input: "\"ab cd\" efgh 'i \" j'"
+ * Output: [ "ab cd", "efgh", "i \" j" ]
+ */
+ static List parseOptionString(String s) {
+ List opts = new ArrayList();
+ StringBuilder opt = new StringBuilder();
+ boolean inOpt = false;
+ boolean inSingleQuote = false;
+ boolean inDoubleQuote = false;
+ boolean escapeNext = false;
+
+ // This is needed to detect when a quoted empty string is used as an argument ("" or '').
+ boolean hasData = false;
+
+ for (int i = 0; i < s.length(); i++) {
+ int c = s.codePointAt(i);
+ if (escapeNext) {
+ opt.appendCodePoint(c);
+ escapeNext = false;
+ } else if (inOpt) {
+ switch (c) {
+ case '\\':
+ if (inSingleQuote) {
+ opt.appendCodePoint(c);
+ } else {
+ escapeNext = true;
+ }
+ break;
+ case '\'':
+ if (inDoubleQuote) {
+ opt.appendCodePoint(c);
+ } else {
+ inSingleQuote = !inSingleQuote;
+ }
+ break;
+ case '"':
+ if (inSingleQuote) {
+ opt.appendCodePoint(c);
+ } else {
+ inDoubleQuote = !inDoubleQuote;
+ }
+ break;
+ default:
+ if (!Character.isWhitespace(c) || inSingleQuote || inDoubleQuote) {
+ opt.appendCodePoint(c);
+ } else {
+ opts.add(opt.toString());
+ opt.setLength(0);
+ inOpt = false;
+ hasData = false;
+ }
+ }
+ } else {
+ switch (c) {
+ case '\'':
+ inSingleQuote = true;
+ inOpt = true;
+ hasData = true;
+ break;
+ case '"':
+ inDoubleQuote = true;
+ inOpt = true;
+ hasData = true;
+ break;
+ case '\\':
+ escapeNext = true;
+ inOpt = true;
+ hasData = true;
+ break;
+ default:
+ if (!Character.isWhitespace(c)) {
+ inOpt = true;
+ hasData = true;
+ opt.appendCodePoint(c);
+ }
+ }
+ }
+ }
+
+ checkArgument(!inSingleQuote && !inDoubleQuote && !escapeNext, "Invalid option string: %s", s);
+ if (hasData) {
+ opts.add(opt.toString());
+ }
+ return opts;
+ }
+
+ /** Throws IllegalArgumentException if the given object is null. */
+ static void checkNotNull(Object o, String arg) {
+ if (o == null) {
+ throw new IllegalArgumentException(String.format("'%s' must not be null.", arg));
+ }
+ }
+
+ /** Throws IllegalArgumentException with the given message if the check is false. */
+ static void checkArgument(boolean check, String msg, Object... args) {
+ if (!check) {
+ throw new IllegalArgumentException(String.format(msg, args));
+ }
+ }
+
+ /** Throws IllegalStateException with the given message if the check is false. */
+ static void checkState(boolean check, String msg, Object... args) {
+ if (!check) {
+ throw new IllegalStateException(String.format(msg, args));
+ }
+ }
+
+ /**
+ * Quote a command argument for a command to be run by a Windows batch script, if the argument
+ * needs quoting. Arguments only seem to need quotes in batch scripts if they have certain
+ * special characters, some of which need extra (and different) escaping.
+ *
+ * For example:
+ * original single argument: ab="cde fgh"
+ * quoted: "ab^=""cde fgh"""
+ */
+ static String quoteForBatchScript(String arg) {
+
+ boolean needsQuotes = false;
+ for (int i = 0; i < arg.length(); i++) {
+ int c = arg.codePointAt(i);
+ if (Character.isWhitespace(c) || c == '"' || c == '=') {
+ needsQuotes = true;
+ break;
+ }
+ }
+ if (!needsQuotes) {
+ return arg;
+ }
+ StringBuilder quoted = new StringBuilder();
+ quoted.append("\"");
+ for (int i = 0; i < arg.length(); i++) {
+ int cp = arg.codePointAt(i);
+ switch (cp) {
+ case '"':
+ quoted.append('"');
+ break;
+
+ case '=':
+ quoted.append('^');
+ break;
+
+ default:
+ break;
+ }
+ quoted.appendCodePoint(cp);
+ }
+ quoted.append("\"");
+ return quoted.toString();
+ }
+
+ /**
+ * Quotes a string so that it can be used in a command string and be parsed back into a single
+ * argument by python's "shlex.split()" function.
+ *
+ * Basically, just add simple escapes. E.g.:
+ * original single argument : ab "cd" ef
+ * after: "ab \"cd\" ef"
+ */
+ static String quoteForPython(String s) {
+ StringBuilder quoted = new StringBuilder().append('"');
+ for (int i = 0; i < s.length(); i++) {
+ int cp = s.codePointAt(i);
+ if (cp == '"' || cp == '\\') {
+ quoted.appendCodePoint('\\');
+ }
+ quoted.appendCodePoint(cp);
+ }
+ return quoted.append('"').toString();
+ }
+
+}
diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java
new file mode 100644
index 0000000000000..206acfb514d86
--- /dev/null
+++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java
@@ -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.launcher;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.spark.launcher.CommandBuilderUtils.*;
+
+/**
+ * Command line interface for the Spark launcher. Used internally by Spark scripts.
+ */
+class Main {
+
+ /**
+ * Usage: Main [class] [class args]
+ *
+ * This CLI works in two different modes:
+ *
+ * - "spark-submit": if class is "org.apache.spark.deploy.SparkSubmit", the
+ * {@link SparkLauncher} class is used to launch a Spark application.
+ * - "spark-class": if another class is provided, an internal Spark class is run.
+ *
+ *
+ * This class works in tandem with the "bin/spark-class" script on Unix-like systems, and
+ * "bin/spark-class2.cmd" batch script on Windows to execute the final command.
+ *
+ * On Unix-like systems, the output is a list of command arguments, separated by the NULL
+ * character. On Windows, the output is a command line suitable for direct execution from the
+ * script.
+ */
+ public static void main(String[] argsArray) throws Exception {
+ checkArgument(argsArray.length > 0, "Not enough arguments: missing class name.");
+
+ List args = new ArrayList(Arrays.asList(argsArray));
+ String className = args.remove(0);
+
+ boolean printLaunchCommand;
+ boolean printUsage;
+ AbstractCommandBuilder builder;
+ try {
+ if (className.equals("org.apache.spark.deploy.SparkSubmit")) {
+ builder = new SparkSubmitCommandBuilder(args);
+ } else {
+ builder = new SparkClassCommandBuilder(className, args);
+ }
+ printLaunchCommand = !isEmpty(System.getenv("SPARK_PRINT_LAUNCH_COMMAND"));
+ printUsage = false;
+ } catch (IllegalArgumentException e) {
+ builder = new UsageCommandBuilder(e.getMessage());
+ printLaunchCommand = false;
+ printUsage = true;
+ }
+
+ Map env = new HashMap();
+ List cmd = builder.buildCommand(env);
+ if (printLaunchCommand) {
+ System.err.println("Spark Command: " + join(" ", cmd));
+ System.err.println("========================================");
+ }
+
+ if (isWindows()) {
+ // When printing the usage message, we can't use "cmd /v" since that prevents the env
+ // variable from being seen in the caller script. So do not call prepareWindowsCommand().
+ if (printUsage) {
+ System.out.println(join(" ", cmd));
+ } else {
+ System.out.println(prepareWindowsCommand(cmd, env));
+ }
+ } else {
+ // In bash, use NULL as the arg separator since it cannot be used in an argument.
+ List bashCmd = prepareBashCommand(cmd, env);
+ for (String c : bashCmd) {
+ System.out.print(c);
+ System.out.print('\0');
+ }
+ }
+ }
+
+ /**
+ * Prepare a command line for execution from a Windows batch script.
+ *
+ * The method quotes all arguments so that spaces are handled as expected. Quotes within arguments
+ * are "double quoted" (which is batch for escaping a quote). This page has more details about
+ * quoting and other batch script fun stuff: http://ss64.com/nt/syntax-esc.html
+ *
+ * The command is executed using "cmd /c" and formatted in single line, since that's the
+ * easiest way to consume this from a batch script (see spark-class2.cmd).
+ */
+ private static String prepareWindowsCommand(List cmd, Map childEnv) {
+ StringBuilder cmdline = new StringBuilder("cmd /c \"");
+ for (Map.Entry e : childEnv.entrySet()) {
+ cmdline.append(String.format("set %s=%s", e.getKey(), e.getValue()));
+ cmdline.append(" && ");
+ }
+ for (String arg : cmd) {
+ cmdline.append(quoteForBatchScript(arg));
+ cmdline.append(" ");
+ }
+ cmdline.append("\"");
+ return cmdline.toString();
+ }
+
+ /**
+ * Prepare the command for execution from a bash script. The final command will have commands to
+ * set up any needed environment variables needed by the child process.
+ */
+ private static List prepareBashCommand(List cmd, Map childEnv) {
+ if (childEnv.isEmpty()) {
+ return cmd;
+ }
+
+ List newCmd = new ArrayList();
+ newCmd.add("env");
+
+ for (Map.Entry e : childEnv.entrySet()) {
+ newCmd.add(String.format("%s=%s", e.getKey(), e.getValue()));
+ }
+ newCmd.addAll(cmd);
+ return newCmd;
+ }
+
+ /**
+ * Internal builder used when command line parsing fails. This will behave differently depending
+ * on the platform:
+ *
+ * - On Unix-like systems, it will print a call to the "usage" function with two arguments: the
+ * the error string, and the exit code to use. The function is expected to print the command's
+ * usage and exit with the provided exit code. The script should use "export -f usage" after
+ * declaring a function called "usage", so that the function is available to downstream scripts.
+ *
+ * - On Windows it will set the variable "SPARK_LAUNCHER_USAGE_ERROR" to the usage error message.
+ * The batch script should check for this variable and print its usage, since batch scripts
+ * don't really support the "export -f" functionality used in bash.
+ */
+ private static class UsageCommandBuilder extends AbstractCommandBuilder {
+
+ private final String message;
+
+ UsageCommandBuilder(String message) {
+ this.message = message;
+ }
+
+ @Override
+ public List buildCommand(Map env) {
+ if (isWindows()) {
+ return Arrays.asList("set", "SPARK_LAUNCHER_USAGE_ERROR=" + message);
+ } else {
+ return Arrays.asList("usage", message, "1");
+ }
+ }
+
+ }
+
+}
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java
new file mode 100644
index 0000000000000..e601a0a19f368
--- /dev/null
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java
@@ -0,0 +1,108 @@
+/*
+ * 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.launcher;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import static org.apache.spark.launcher.CommandBuilderUtils.*;
+
+/**
+ * Command builder for internal Spark classes.
+ *
+ * This class handles building the command to launch all internal Spark classes except for
+ * SparkSubmit (which is handled by {@link SparkSubmitCommandBuilder} class.
+ */
+class SparkClassCommandBuilder extends AbstractCommandBuilder {
+
+ private final String className;
+ private final List classArgs;
+
+ SparkClassCommandBuilder(String className, List classArgs) {
+ this.className = className;
+ this.classArgs = classArgs;
+ }
+
+ @Override
+ public List buildCommand(Map env) throws IOException {
+ List javaOptsKeys = new ArrayList();
+ String memKey = null;
+ String extraClassPath = null;
+
+ // Master, Worker, and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) +
+ // SPARK_DAEMON_MEMORY.
+ if (className.equals("org.apache.spark.deploy.master.Master")) {
+ javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
+ javaOptsKeys.add("SPARK_MASTER_OPTS");
+ memKey = "SPARK_DAEMON_MEMORY";
+ } else if (className.equals("org.apache.spark.deploy.worker.Worker")) {
+ javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
+ javaOptsKeys.add("SPARK_WORKER_OPTS");
+ memKey = "SPARK_DAEMON_MEMORY";
+ } else if (className.equals("org.apache.spark.deploy.history.HistoryServer")) {
+ javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
+ javaOptsKeys.add("SPARK_HISTORY_OPTS");
+ memKey = "SPARK_DAEMON_MEMORY";
+ } else if (className.equals("org.apache.spark.executor.CoarseGrainedExecutorBackend")) {
+ javaOptsKeys.add("SPARK_JAVA_OPTS");
+ javaOptsKeys.add("SPARK_EXECUTOR_OPTS");
+ memKey = "SPARK_EXECUTOR_MEMORY";
+ } else if (className.equals("org.apache.spark.executor.MesosExecutorBackend")) {
+ javaOptsKeys.add("SPARK_EXECUTOR_OPTS");
+ memKey = "SPARK_EXECUTOR_MEMORY";
+ } else if (className.startsWith("org.apache.spark.tools.")) {
+ String sparkHome = getSparkHome();
+ File toolsDir = new File(join(File.separator, sparkHome, "tools", "target",
+ "scala-" + getScalaVersion()));
+ checkState(toolsDir.isDirectory(), "Cannot find tools build directory.");
+
+ Pattern re = Pattern.compile("spark-tools_.*\\.jar");
+ for (File f : toolsDir.listFiles()) {
+ if (re.matcher(f.getName()).matches()) {
+ extraClassPath = f.getAbsolutePath();
+ break;
+ }
+ }
+
+ checkState(extraClassPath != null,
+ "Failed to find Spark Tools Jar in %s.\n" +
+ "You need to run \"build/sbt tools/package\" before running %s.",
+ toolsDir.getAbsolutePath(), className);
+
+ javaOptsKeys.add("SPARK_JAVA_OPTS");
+ }
+
+ List cmd = buildJavaCommand(extraClassPath);
+ for (String key : javaOptsKeys) {
+ addOptionString(cmd, System.getenv(key));
+ }
+
+ String mem = firstNonEmpty(memKey != null ? System.getenv(memKey) : null, DEFAULT_MEM);
+ cmd.add("-Xms" + mem);
+ cmd.add("-Xmx" + mem);
+ addPermGenSizeOpt(cmd);
+ cmd.add(className);
+ cmd.addAll(classArgs);
+ return cmd;
+ }
+
+}
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
new file mode 100644
index 0000000000000..b566507ee6061
--- /dev/null
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
@@ -0,0 +1,279 @@
+/*
+ * 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.launcher;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.spark.launcher.CommandBuilderUtils.*;
+
+/**
+ * Launcher for Spark applications.
+ *
+ * Use this class to start Spark applications programmatically. The class uses a builder pattern
+ * to allow clients to configure the Spark application and launch it as a child process.
+ */
+public class SparkLauncher {
+
+ /** The Spark master. */
+ public static final String SPARK_MASTER = "spark.master";
+
+ /** Configuration key for the driver memory. */
+ public static final String DRIVER_MEMORY = "spark.driver.memory";
+ /** Configuration key for the driver class path. */
+ public static final String DRIVER_EXTRA_CLASSPATH = "spark.driver.extraClassPath";
+ /** Configuration key for the driver VM options. */
+ public static final String DRIVER_EXTRA_JAVA_OPTIONS = "spark.driver.extraJavaOptions";
+ /** Configuration key for the driver native library path. */
+ public static final String DRIVER_EXTRA_LIBRARY_PATH = "spark.driver.extraLibraryPath";
+
+ /** Configuration key for the executor memory. */
+ public static final String EXECUTOR_MEMORY = "spark.executor.memory";
+ /** Configuration key for the executor class path. */
+ public static final String EXECUTOR_EXTRA_CLASSPATH = "spark.executor.extraClassPath";
+ /** Configuration key for the executor VM options. */
+ public static final String EXECUTOR_EXTRA_JAVA_OPTIONS = "spark.executor.extraJavaOptions";
+ /** Configuration key for the executor native library path. */
+ public static final String EXECUTOR_EXTRA_LIBRARY_PATH = "spark.executor.extraLibraryOptions";
+ /** Configuration key for the number of executor CPU cores. */
+ public static final String EXECUTOR_CORES = "spark.executor.cores";
+
+ private final SparkSubmitCommandBuilder builder;
+
+ public SparkLauncher() {
+ this(null);
+ }
+
+ /**
+ * Creates a launcher that will set the given environment variables in the child.
+ *
+ * @param env Environment variables to set.
+ */
+ public SparkLauncher(Map env) {
+ this.builder = new SparkSubmitCommandBuilder();
+ if (env != null) {
+ this.builder.childEnv.putAll(env);
+ }
+ }
+
+ /**
+ * Set a custom JAVA_HOME for launching the Spark application.
+ *
+ * @param javaHome Path to the JAVA_HOME to use.
+ * @return This launcher.
+ */
+ public SparkLauncher setJavaHome(String javaHome) {
+ checkNotNull(javaHome, "javaHome");
+ builder.javaHome = javaHome;
+ return this;
+ }
+
+ /**
+ * Set a custom Spark installation location for the application.
+ *
+ * @param sparkHome Path to the Spark installation to use.
+ * @return This launcher.
+ */
+ public SparkLauncher setSparkHome(String sparkHome) {
+ checkNotNull(sparkHome, "sparkHome");
+ builder.childEnv.put(ENV_SPARK_HOME, sparkHome);
+ return this;
+ }
+
+ /**
+ * Set a custom properties file with Spark configuration for the application.
+ *
+ * @param path Path to custom properties file to use.
+ * @return This launcher.
+ */
+ public SparkLauncher setPropertiesFile(String path) {
+ checkNotNull(path, "path");
+ builder.propertiesFile = path;
+ return this;
+ }
+
+ /**
+ * Set a single configuration value for the application.
+ *
+ * @param key Configuration key.
+ * @param value The value to use.
+ * @return This launcher.
+ */
+ public SparkLauncher setConf(String key, String value) {
+ checkNotNull(key, "key");
+ checkNotNull(value, "value");
+ checkArgument(key.startsWith("spark."), "'key' must start with 'spark.'");
+ builder.conf.put(key, value);
+ return this;
+ }
+
+ /**
+ * Set the application name.
+ *
+ * @param appName Application name.
+ * @return This launcher.
+ */
+ public SparkLauncher setAppName(String appName) {
+ checkNotNull(appName, "appName");
+ builder.appName = appName;
+ return this;
+ }
+
+ /**
+ * Set the Spark master for the application.
+ *
+ * @param master Spark master.
+ * @return This launcher.
+ */
+ public SparkLauncher setMaster(String master) {
+ checkNotNull(master, "master");
+ builder.master = master;
+ return this;
+ }
+
+ /**
+ * Set the deploy mode for the application.
+ *
+ * @param mode Deploy mode.
+ * @return This launcher.
+ */
+ public SparkLauncher setDeployMode(String mode) {
+ checkNotNull(mode, "mode");
+ builder.deployMode = mode;
+ return this;
+ }
+
+ /**
+ * Set the main application resource. This should be the location of a jar file for Scala/Java
+ * applications, or a python script for PySpark applications.
+ *
+ * @param resource Path to the main application resource.
+ * @return This launcher.
+ */
+ public SparkLauncher setAppResource(String resource) {
+ checkNotNull(resource, "resource");
+ builder.appResource = resource;
+ return this;
+ }
+
+ /**
+ * Sets the application class name for Java/Scala applications.
+ *
+ * @param mainClass Application's main class.
+ * @return This launcher.
+ */
+ public SparkLauncher setMainClass(String mainClass) {
+ checkNotNull(mainClass, "mainClass");
+ builder.mainClass = mainClass;
+ return this;
+ }
+
+ /**
+ * Adds command line arguments for the application.
+ *
+ * @param args Arguments to pass to the application's main class.
+ * @return This launcher.
+ */
+ public SparkLauncher addAppArgs(String... args) {
+ for (String arg : args) {
+ checkNotNull(arg, "arg");
+ builder.appArgs.add(arg);
+ }
+ return this;
+ }
+
+ /**
+ * Adds a jar file to be submitted with the application.
+ *
+ * @param jar Path to the jar file.
+ * @return This launcher.
+ */
+ public SparkLauncher addJar(String jar) {
+ checkNotNull(jar, "jar");
+ builder.jars.add(jar);
+ return this;
+ }
+
+ /**
+ * Adds a file to be submitted with the application.
+ *
+ * @param file Path to the file.
+ * @return This launcher.
+ */
+ public SparkLauncher addFile(String file) {
+ checkNotNull(file, "file");
+ builder.files.add(file);
+ return this;
+ }
+
+ /**
+ * Adds a python file / zip / egg to be submitted with the application.
+ *
+ * @param file Path to the file.
+ * @return This launcher.
+ */
+ public SparkLauncher addPyFile(String file) {
+ checkNotNull(file, "file");
+ builder.pyFiles.add(file);
+ return this;
+ }
+
+ /**
+ * Enables verbose reporting for SparkSubmit.
+ *
+ * @param verbose Whether to enable verbose output.
+ * @return This launcher.
+ */
+ public SparkLauncher setVerbose(boolean verbose) {
+ builder.verbose = verbose;
+ return this;
+ }
+
+ /**
+ * Launches a sub-process that will start the configured Spark application.
+ *
+ * @return A process handle for the Spark app.
+ */
+ public Process launch() throws IOException {
+ List cmd = new ArrayList();
+ String script = isWindows() ? "spark-submit.cmd" : "spark-submit";
+ cmd.add(join(File.separator, builder.getSparkHome(), "bin", script));
+ cmd.addAll(builder.buildSparkSubmitArgs());
+
+ // Since the child process is a batch script, let's quote things so that special characters are
+ // preserved, otherwise the batch interpreter will mess up the arguments. Batch scripts are
+ // weird.
+ if (isWindows()) {
+ List winCmd = new ArrayList();
+ for (String arg : cmd) {
+ winCmd.add(quoteForBatchScript(arg));
+ }
+ cmd = winCmd;
+ }
+
+ ProcessBuilder pb = new ProcessBuilder(cmd.toArray(new String[cmd.size()]));
+ for (Map.Entry e : builder.childEnv.entrySet()) {
+ pb.environment().put(e.getKey(), e.getValue());
+ }
+ return pb.start();
+ }
+
+}
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
new file mode 100644
index 0000000000000..6ffdff63d3c78
--- /dev/null
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
@@ -0,0 +1,327 @@
+/*
+ * 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.launcher;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.spark.launcher.CommandBuilderUtils.*;
+
+/**
+ * Special command builder for handling a CLI invocation of SparkSubmit.
+ *
+ * This builder adds command line parsing compatible with SparkSubmit. It handles setting
+ * driver-side options and special parsing behavior needed for the special-casing certain internal
+ * Spark applications.
+ *
+ * This class has also some special features to aid launching pyspark.
+ */
+class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
+
+ /**
+ * Name of the app resource used to identify the PySpark shell. The command line parser expects
+ * the resource name to be the very first argument to spark-submit in this case.
+ *
+ * NOTE: this cannot be "pyspark-shell" since that identifies the PySpark shell to SparkSubmit
+ * (see java_gateway.py), and can cause this code to enter into an infinite loop.
+ */
+ static final String PYSPARK_SHELL = "pyspark-shell-main";
+
+ /**
+ * This is the actual resource name that identifies the PySpark shell to SparkSubmit.
+ */
+ static final String PYSPARK_SHELL_RESOURCE = "pyspark-shell";
+
+ /**
+ * This map must match the class names for available special classes, since this modifies the way
+ * command line parsing works. This maps the class name to the resource to use when calling
+ * spark-submit.
+ */
+ private static final Map specialClasses = new HashMap();
+ static {
+ specialClasses.put("org.apache.spark.repl.Main", "spark-shell");
+ specialClasses.put("org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver",
+ "spark-internal");
+ specialClasses.put("org.apache.spark.sql.hive.thriftserver.HiveThriftServer2",
+ "spark-internal");
+ }
+
+ private final List sparkArgs;
+
+ /**
+ * Controls whether mixing spark-submit arguments with app arguments is allowed. This is needed
+ * to parse the command lines for things like bin/spark-shell, which allows users to mix and
+ * match arguments (e.g. "bin/spark-shell SparkShellArg --master foo").
+ */
+ private boolean allowsMixedArguments;
+
+ SparkSubmitCommandBuilder() {
+ this.sparkArgs = new ArrayList();
+ }
+
+ SparkSubmitCommandBuilder(List args) {
+ this();
+ List submitArgs = args;
+ if (args.size() > 0 && args.get(0).equals(PYSPARK_SHELL)) {
+ this.allowsMixedArguments = true;
+ appResource = PYSPARK_SHELL_RESOURCE;
+ submitArgs = args.subList(1, args.size());
+ } else {
+ this.allowsMixedArguments = false;
+ }
+
+ new OptionParser().parse(submitArgs);
+ }
+
+ @Override
+ public List buildCommand(Map env) throws IOException {
+ if (PYSPARK_SHELL_RESOURCE.equals(appResource)) {
+ return buildPySparkShellCommand(env);
+ } else {
+ return buildSparkSubmitCommand(env);
+ }
+ }
+
+ List buildSparkSubmitArgs() {
+ List args = new ArrayList();
+ SparkSubmitOptionParser parser = new SparkSubmitOptionParser();
+
+ if (verbose) {
+ args.add(parser.VERBOSE);
+ }
+
+ if (master != null) {
+ args.add(parser.MASTER);
+ args.add(master);
+ }
+
+ if (deployMode != null) {
+ args.add(parser.DEPLOY_MODE);
+ args.add(deployMode);
+ }
+
+ if (appName != null) {
+ args.add(parser.NAME);
+ args.add(appName);
+ }
+
+ for (Map.Entry e : conf.entrySet()) {
+ args.add(parser.CONF);
+ args.add(String.format("%s=%s", e.getKey(), e.getValue()));
+ }
+
+ if (propertiesFile != null) {
+ args.add(parser.PROPERTIES_FILE);
+ args.add(propertiesFile);
+ }
+
+ if (!jars.isEmpty()) {
+ args.add(parser.JARS);
+ args.add(join(",", jars));
+ }
+
+ if (!files.isEmpty()) {
+ args.add(parser.FILES);
+ args.add(join(",", files));
+ }
+
+ if (!pyFiles.isEmpty()) {
+ args.add(parser.PY_FILES);
+ args.add(join(",", pyFiles));
+ }
+
+ if (mainClass != null) {
+ args.add(parser.CLASS);
+ args.add(mainClass);
+ }
+
+ args.addAll(sparkArgs);
+ if (appResource != null) {
+ args.add(appResource);
+ }
+ args.addAll(appArgs);
+
+ return args;
+ }
+
+ private List buildSparkSubmitCommand(Map env) throws IOException {
+ // Load the properties file and check whether spark-submit will be running the app's driver
+ // or just launching a cluster app. When running the driver, the JVM's argument will be
+ // modified to cover the driver's configuration.
+ Properties props = loadPropertiesFile();
+ boolean isClientMode = isClientMode(props);
+ String extraClassPath = isClientMode ?
+ firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_CLASSPATH, conf, props) : null;
+
+ List cmd = buildJavaCommand(extraClassPath);
+ addOptionString(cmd, System.getenv("SPARK_SUBMIT_OPTS"));
+ addOptionString(cmd, System.getenv("SPARK_JAVA_OPTS"));
+
+ if (isClientMode) {
+ // Figuring out where the memory value come from is a little tricky due to precedence.
+ // Precedence is observed in the following order:
+ // - explicit configuration (setConf()), which also covers --driver-memory cli argument.
+ // - properties file.
+ // - SPARK_DRIVER_MEMORY env variable
+ // - SPARK_MEM env variable
+ // - default value (512m)
+ String memory = firstNonEmpty(firstNonEmptyValue(SparkLauncher.DRIVER_MEMORY, conf, props),
+ System.getenv("SPARK_DRIVER_MEMORY"), System.getenv("SPARK_MEM"), DEFAULT_MEM);
+ cmd.add("-Xms" + memory);
+ cmd.add("-Xmx" + memory);
+ addOptionString(cmd, firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, conf, props));
+ mergeEnvPathList(env, getLibPathEnvName(),
+ firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props));
+ }
+
+ addPermGenSizeOpt(cmd);
+ cmd.add("org.apache.spark.deploy.SparkSubmit");
+ cmd.addAll(buildSparkSubmitArgs());
+ return cmd;
+ }
+
+ private List buildPySparkShellCommand(Map env) throws IOException {
+ // For backwards compatibility, if a script is specified in
+ // the pyspark command line, then run it using spark-submit.
+ if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".py")) {
+ System.err.println(
+ "WARNING: Running python applications through 'pyspark' is deprecated as of Spark 1.0.\n" +
+ "Use ./bin/spark-submit ");
+ appResource = appArgs.get(0);
+ appArgs.remove(0);
+ return buildCommand(env);
+ }
+
+ // When launching the pyspark shell, the spark-submit arguments should be stored in the
+ // PYSPARK_SUBMIT_ARGS env variable. The executable is the PYSPARK_DRIVER_PYTHON env variable
+ // set by the pyspark script, followed by PYSPARK_DRIVER_PYTHON_OPTS.
+ checkArgument(appArgs.isEmpty(), "pyspark does not support any application options.");
+
+ Properties props = loadPropertiesFile();
+ mergeEnvPathList(env, getLibPathEnvName(),
+ firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props));
+
+ // Store spark-submit arguments in an environment variable, since there's no way to pass
+ // them to shell.py on the comand line.
+ StringBuilder submitArgs = new StringBuilder();
+ for (String arg : buildSparkSubmitArgs()) {
+ if (submitArgs.length() > 0) {
+ submitArgs.append(" ");
+ }
+ submitArgs.append(quoteForPython(arg));
+ }
+ env.put("PYSPARK_SUBMIT_ARGS", submitArgs.toString());
+
+ List pyargs = new ArrayList();
+ pyargs.add(firstNonEmpty(System.getenv("PYSPARK_DRIVER_PYTHON"), "python"));
+ String pyOpts = System.getenv("PYSPARK_DRIVER_PYTHON_OPTS");
+ if (!isEmpty(pyOpts)) {
+ pyargs.addAll(parseOptionString(pyOpts));
+ }
+
+ return pyargs;
+ }
+
+ private boolean isClientMode(Properties userProps) {
+ String userMaster = firstNonEmpty(master, (String) userProps.get(SparkLauncher.SPARK_MASTER));
+ // Default master is "local[*]", so assume client mode in that case.
+ return userMaster == null ||
+ "client".equals(deployMode) ||
+ (!userMaster.equals("yarn-cluster") && deployMode == null);
+ }
+
+ private class OptionParser extends SparkSubmitOptionParser {
+
+ private final List
- URL: {state.uri} + { + state.restUri.map { uri => +
- + REST URL: {uri} + (cluster mode) + + }.getOrElse { Seq.empty } + }
- Workers: {state.workers.size}
- Cores: {state.workers.map(_.cores).sum} Total, {state.workers.map(_.coresUsed).sum} Used @@ -154,16 +191,34 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { } - private def appRow(app: ApplicationInfo): Seq[Node] = { + private def appRow(app: ApplicationInfo, active: Boolean): Seq[Node] = { + val killLink = if (parent.killEnabled && + (app.state == ApplicationState.RUNNING || app.state == ApplicationState.WAITING)) { + val killLinkUri = s"app/kill?id=${app.id}&terminate=true" + val confirm = "return window.confirm(" + + s"'Are you sure you want to kill application ${app.id} ?');" + + (kill) + + } +
- - Input: - {Utils.bytesToString(stageData.inputBytes)} + Input Size / Records: + {s"${Utils.bytesToString(stageData.inputBytes)} / ${stageData.inputRecords}"} }} - {if (hasOutput) { + {if (stageData.hasOutput) {
- Output: - {Utils.bytesToString(stageData.outputBytes)} + {s"${Utils.bytesToString(stageData.outputBytes)} / ${stageData.outputRecords}"} }} - {if (hasShuffleRead) { + {if (stageData.hasShuffleRead) {
- Shuffle read: - {Utils.bytesToString(stageData.shuffleReadBytes)} + {s"${Utils.bytesToString(stageData.shuffleReadTotalBytes)} / " + + s"${stageData.shuffleReadRecords}"} }} - {if (hasShuffleWrite) { + {if (stageData.hasShuffleWrite) {
- Shuffle write: - {Utils.bytesToString(stageData.shuffleWriteBytes)} + {s"${Utils.bytesToString(stageData.shuffleWriteBytes)} / " + + s"${stageData.shuffleWriteRecords}"} }} - {if (hasBytesSpilled) { + {if (stageData.hasBytesSpilled) {
- Shuffle spill (memory): {Utils.bytesToString(stageData.memoryBytesSpilled)} @@ -132,7 +135,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { Task Deserialization Time - {if (hasShuffleRead) { + {if (stageData.hasShuffleRead) {
- @@ -140,6 +143,13 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { Shuffle Read Blocked Time +
- + + + Shuffle Remote Reads + + }}
-
+ getDistributionQuantiles(times).map { millis =>
{UIUtils.formatDuration(millis.toLong)} } } @@ -273,35 +294,86 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { getFormattedTimeQuantiles(schedulerDelays) def getFormattedSizeQuantiles(data: Seq[Double]) = - Distribution(data).get.getQuantiles().map(d =>{Utils.bytesToString(d.toLong)} ) + getDistributionQuantiles(data).map(d =>{Utils.bytesToString(d.toLong)} ) + + def getFormattedSizeQuantilesWithRecords(data: Seq[Double], records: Seq[Double]) = { + val recordDist = getDistributionQuantiles(records).iterator + getDistributionQuantiles(data).map(d => +{s"${Utils.bytesToString(d.toLong)} / ${recordDist.next().toLong}"} + ) + } val inputSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.inputMetrics.map(_.bytesRead).getOrElse(0L).toDouble } - val inputQuantiles =Input +: getFormattedSizeQuantiles(inputSizes) + + val inputRecords = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.inputMetrics.map(_.recordsRead).getOrElse(0L).toDouble + } + + val inputQuantiles =Input Size / Records +: + getFormattedSizeQuantilesWithRecords(inputSizes, inputRecords) val outputSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.outputMetrics.map(_.bytesWritten).getOrElse(0L).toDouble } - val outputQuantiles =Output +: getFormattedSizeQuantiles(outputSizes) + + val outputRecords = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.outputMetrics.map(_.recordsWritten).getOrElse(0L).toDouble + } + + val outputQuantiles =Output Size / Records +: + getFormattedSizeQuantilesWithRecords(outputSizes, outputRecords) val shuffleReadBlockedTimes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.shuffleReadMetrics.map(_.fetchWaitTime).getOrElse(0L).toDouble } - val shuffleReadBlockedQuantiles =Shuffle Read Blocked Time +: + val shuffleReadBlockedQuantiles = ++ + Shuffle Read Blocked Time + + +: getFormattedTimeQuantiles(shuffleReadBlockedTimes) - val shuffleReadSizes = validTasks.map { case TaskUIData(_, metrics, _) => + val shuffleReadTotalSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.shuffleReadMetrics.map(_.totalBytesRead).getOrElse(0L).toDouble + } + val shuffleReadTotalRecords = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.shuffleReadMetrics.map(_.recordsRead).getOrElse(0L).toDouble + } + val shuffleReadTotalQuantiles = ++ + Shuffle Read Size / Records + + +: + getFormattedSizeQuantilesWithRecords(shuffleReadTotalSizes, shuffleReadTotalRecords) + + val shuffleReadRemoteSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble } - val shuffleReadQuantiles =Shuffle Read (Remote) +: - getFormattedSizeQuantiles(shuffleReadSizes) + val shuffleReadRemoteQuantiles = ++ + Shuffle Remote Reads + + +: + getFormattedSizeQuantiles(shuffleReadRemoteSizes) val shuffleWriteSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble } - val shuffleWriteQuantiles =Shuffle Write +: - getFormattedSizeQuantiles(shuffleWriteSizes) + + val shuffleWriteRecords = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.shuffleWriteMetrics.map(_.shuffleRecordsWritten).getOrElse(0L).toDouble + } + + val shuffleWriteQuantiles =Shuffle Write Size / Records +: + getFormattedSizeQuantilesWithRecords(shuffleWriteSizes, shuffleWriteRecords) val memoryBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.memoryBytesSpilled.toDouble @@ -326,19 +398,22 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {serializationQuantiles} ,{gettingResultQuantiles} , - if (hasInput){inputQuantiles} else Nil, - if (hasOutput){outputQuantiles} else Nil, - if (hasShuffleRead) { + if (stageData.hasInput){inputQuantiles} else Nil, + if (stageData.hasOutput){outputQuantiles} else Nil, + if (stageData.hasShuffleRead) {{shuffleReadBlockedQuantiles} -{shuffleReadQuantiles} +{shuffleReadTotalQuantiles} ++ {shuffleReadRemoteQuantiles} + } else { Nil }, - if (hasShuffleWrite){shuffleWriteQuantiles} else Nil, - if (hasBytesSpilled){memoryBytesSpilledQuantiles} else Nil, - if (hasBytesSpilled){diskBytesSpilledQuantiles} else Nil) + if (stageData.hasShuffleWrite){shuffleWriteQuantiles} else Nil, + if (stageData.hasBytesSpilled){memoryBytesSpilledQuantiles} else Nil, + if (stageData.hasBytesSpilled){diskBytesSpilledQuantiles} else Nil) val quantileHeaders = Seq("Metric", "Min", "25th percentile", "Median", "75th percentile", "Max") @@ -397,26 +472,36 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val inputReadable = maybeInput .map(m => s"${Utils.bytesToString(m.bytesRead)} (${m.readMethod.toString.toLowerCase()})") .getOrElse("") + val inputRecords = maybeInput.map(_.recordsRead.toString).getOrElse("") val maybeOutput = metrics.flatMap(_.outputMetrics) val outputSortable = maybeOutput.map(_.bytesWritten.toString).getOrElse("") val outputReadable = maybeOutput .map(m => s"${Utils.bytesToString(m.bytesWritten)}") .getOrElse("") + val outputRecords = maybeOutput.map(_.recordsWritten.toString).getOrElse("") - val maybeShuffleReadBlockedTime = metrics.flatMap(_.shuffleReadMetrics).map(_.fetchWaitTime) - val shuffleReadBlockedTimeSortable = maybeShuffleReadBlockedTime.map(_.toString).getOrElse("") + val maybeShuffleRead = metrics.flatMap(_.shuffleReadMetrics) + val shuffleReadBlockedTimeSortable = maybeShuffleRead + .map(_.fetchWaitTime.toString).getOrElse("") val shuffleReadBlockedTimeReadable = - maybeShuffleReadBlockedTime.map(ms => UIUtils.formatDuration(ms)).getOrElse("") + maybeShuffleRead.map(ms => UIUtils.formatDuration(ms.fetchWaitTime)).getOrElse("") + + val totalShuffleBytes = maybeShuffleRead.map(_.totalBytesRead) + val shuffleReadSortable = totalShuffleBytes.map(_.toString).getOrElse("") + val shuffleReadReadable = totalShuffleBytes.map(Utils.bytesToString).getOrElse("") + val shuffleReadRecords = maybeShuffleRead.map(_.recordsRead.toString).getOrElse("") - val maybeShuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead) - val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") - val shuffleReadReadable = maybeShuffleRead.map(Utils.bytesToString).getOrElse("") + val remoteShuffleBytes = maybeShuffleRead.map(_.remoteBytesRead) + val shuffleReadRemoteSortable = remoteShuffleBytes.map(_.toString).getOrElse("") + val shuffleReadRemoteReadable = remoteShuffleBytes.map(Utils.bytesToString).getOrElse("") - val maybeShuffleWrite = - metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten) - val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("") - val shuffleWriteReadable = maybeShuffleWrite.map(Utils.bytesToString).getOrElse("") + val maybeShuffleWrite = metrics.flatMap(_.shuffleWriteMetrics) + val shuffleWriteSortable = maybeShuffleWrite.map(_.shuffleBytesWritten.toString).getOrElse("") + val shuffleWriteReadable = maybeShuffleWrite + .map(m => s"${Utils.bytesToString(m.shuffleBytesWritten)}").getOrElse("") + val shuffleWriteRecords = maybeShuffleWrite + .map(_.shuffleRecordsWritten.toString).getOrElse("") val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") @@ -472,12 +557,12 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { }} {if (hasInput) {- {inputReadable} + {s"$inputReadable / $inputRecords"} }} {if (hasOutput) {- {outputReadable} + {s"$outputReadable / $outputRecords"} }} {if (hasShuffleRead) { @@ -486,7 +571,11 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {shuffleReadBlockedTimeReadable}- {shuffleReadReadable} + {s"$shuffleReadReadable / $shuffleReadRecords"} + ++ {shuffleReadRemoteReadable} }} {if (hasShuffleWrite) { @@ -494,7 +583,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {writeTimeReadable}- {shuffleWriteReadable} + {s"$shuffleWriteReadable / $shuffleWriteRecords"} }} {if (hasBytesSpilled) { @@ -537,15 +626,16 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { } private def getSchedulerDelay(info: TaskInfo, metrics: TaskMetrics): Long = { - val totalExecutionTime = { - if (info.gettingResultTime > 0) { - (info.gettingResultTime - info.launchTime) + val totalExecutionTime = + if (info.gettingResult) { + info.gettingResultTime - info.launchTime + } else if (info.finished) { + info.finishTime - info.launchTime } else { - (info.finishTime - info.launchTime) + 0 } - } val executorOverhead = (metrics.executorDeserializeTime + metrics.resultSerializationTime) - totalExecutionTime - metrics.executorRunTime - executorOverhead + math.max(0, totalExecutionTime - metrics.executorRunTime - executorOverhead) } } 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 703d43f9c640d..5865850fa09b5 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 @@ -138,7 +138,7 @@ private[ui] class StageTableBase( val inputReadWithUnit = if (inputRead > 0) Utils.bytesToString(inputRead) else "" val outputWrite = stageData.outputBytes val outputWriteWithUnit = if (outputWrite > 0) Utils.bytesToString(outputWrite) else "" - val shuffleRead = stageData.shuffleReadBytes + val shuffleRead = stageData.shuffleReadTotalBytes val shuffleReadWithUnit = if (shuffleRead > 0) Utils.bytesToString(shuffleRead) else "" val shuffleWrite = stageData.shuffleWriteBytes val shuffleWriteWithUnit = if (shuffleWrite > 0) Utils.bytesToString(shuffleWrite) else "" diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala index 37cf2c207ba40..9bf67db8acde1 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala @@ -28,6 +28,7 @@ private[spark] object TaskDetailsClassNames { val SCHEDULER_DELAY = "scheduler_delay" val TASK_DESERIALIZATION_TIME = "deserialization_time" val SHUFFLE_READ_BLOCKED_TIME = "fetch_wait_time" + val SHUFFLE_READ_REMOTE_SIZE = "shuffle_read_remote" val RESULT_SERIALIZATION_TIME = "serialization_time" val GETTING_RESULT_TIME = "getting_result_time" } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 01f7e23212c3d..dbf1ceeda1878 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -31,9 +31,13 @@ private[jobs] object UIData { var failedTasks : Int = 0 var succeededTasks : Int = 0 var inputBytes : Long = 0 + var inputRecords : Long = 0 var outputBytes : Long = 0 + var outputRecords : Long = 0 var shuffleRead : Long = 0 + var shuffleReadRecords : Long = 0 var shuffleWrite : Long = 0 + var shuffleWriteRecords : Long = 0 var memoryBytesSpilled : Long = 0 var diskBytesSpilled : Long = 0 } @@ -73,9 +77,13 @@ private[jobs] object UIData { var executorRunTime: Long = _ var inputBytes: Long = _ + var inputRecords: Long = _ var outputBytes: Long = _ - var shuffleReadBytes: Long = _ + var outputRecords: Long = _ + var shuffleReadTotalBytes: Long = _ + var shuffleReadRecords : Long = _ var shuffleWriteBytes: Long = _ + var shuffleWriteRecords: Long = _ var memoryBytesSpilled: Long = _ var diskBytesSpilled: Long = _ @@ -85,6 +93,12 @@ private[jobs] object UIData { var accumulables = new HashMap[Long, AccumulableInfo] var taskData = new HashMap[Long, TaskUIData] var executorSummary = new HashMap[String, ExecutorSummary] + + def hasInput = inputBytes > 0 + def hasOutput = outputBytes > 0 + def hasShuffleRead = shuffleReadTotalBytes > 0 + def hasShuffleWrite = shuffleWriteBytes > 0 + def hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0 } /** diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 12d23a92878cf..199f731b92bcc 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -30,7 +30,10 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - val rddId = request.getParameter("id").toInt + val parameterId = request.getParameter("id") + require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") + + val rddId = parameterId.toInt val storageStatusList = listener.storageStatusList val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse { // Rather than crashing, render an "RDD Not Found" page 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 4c9b1e3c46f0f..48a6ede05e17b 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.util import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.Await import scala.concurrent.duration.{Duration, FiniteDuration} +import scala.util.Try import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem} import akka.pattern.ask @@ -78,8 +79,6 @@ private[spark] object AkkaUtils extends Logging { val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 6000) - val akkaFailureDetector = - conf.getDouble("spark.akka.failure-detector.threshold", 300.0) val akkaHeartBeatInterval = conf.getInt("spark.akka.heartbeat.interval", 1000) val secretKey = securityManager.getSecretKey() @@ -91,8 +90,11 @@ private[spark] object AkkaUtils extends Logging { val secureCookie = if (isAuthOn) secretKey else "" logDebug(s"In createActorSystem, requireCookie is: $requireCookie") - val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]).withFallback( - ConfigFactory.parseString( + val akkaSslConfig = securityManager.akkaSSLOptions.createAkkaConfig + .getOrElse(ConfigFactory.empty()) + + val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]) + .withFallback(akkaSslConfig).withFallback(ConfigFactory.parseString( s""" |akka.daemonic = on |akka.loggers = [""akka.event.slf4j.Slf4jLogger""] @@ -102,7 +104,6 @@ private[spark] object AkkaUtils extends Logging { |akka.remote.secure-cookie = "$secureCookie" |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s - |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector |akka.actor.provider = "akka.remote.RemoteActorRefProvider" |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" |akka.remote.netty.tcp.hostname = "$host" @@ -214,7 +215,7 @@ private[spark] object AkkaUtils extends Logging { val driverHost: String = conf.get("spark.driver.host", "localhost") val driverPort: Int = conf.getInt("spark.driver.port", 7077) Utils.checkHost(driverHost, "Expected hostname") - val url = s"akka.tcp://$driverActorSystemName@$driverHost:$driverPort/user/$name" + val url = address(protocol(actorSystem), driverActorSystemName, driverHost, driverPort, name) val timeout = AkkaUtils.lookupTimeout(conf) logInfo(s"Connecting to $name: $url") Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) @@ -228,9 +229,33 @@ private[spark] object AkkaUtils extends Logging { actorSystem: ActorSystem): ActorRef = { val executorActorSystemName = SparkEnv.executorActorSystemName Utils.checkHost(host, "Expected hostname") - val url = s"akka.tcp://$executorActorSystemName@$host:$port/user/$name" + val url = address(protocol(actorSystem), executorActorSystemName, host, port, name) val timeout = AkkaUtils.lookupTimeout(conf) logInfo(s"Connecting to $name: $url") Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) } + + def protocol(actorSystem: ActorSystem): String = { + val akkaConf = actorSystem.settings.config + val sslProp = "akka.remote.netty.tcp.enable-ssl" + protocol(akkaConf.hasPath(sslProp) && akkaConf.getBoolean(sslProp)) + } + + def protocol(ssl: Boolean = false): String = { + if (ssl) { + "akka.ssl.tcp" + } else { + "akka.tcp" + } + } + + def address( + protocol: String, + systemName: String, + host: String, + port: Any, + actorName: String): String = { + s"$protocol://$systemName@$host:$port/user/$actorName" + } + } diff --git a/core/src/main/scala/org/apache/spark/util/Clock.scala b/core/src/main/scala/org/apache/spark/util/Clock.scala index 97c2b45aabf28..e92ed11bd165b 100644 --- a/core/src/main/scala/org/apache/spark/util/Clock.scala +++ b/core/src/main/scala/org/apache/spark/util/Clock.scala @@ -21,9 +21,47 @@ package org.apache.spark.util * An interface to represent clocks, so that they can be mocked out in unit tests. */ private[spark] trait Clock { - def getTime(): Long + def getTimeMillis(): Long + def waitTillTime(targetTime: Long): Long } -private[spark] object SystemClock extends Clock { - def getTime(): Long = System.currentTimeMillis() +/** + * A clock backed by the actual time from the OS as reported by the `System` API. + */ +private[spark] class SystemClock extends Clock { + + val minPollTime = 25L + + /** + * @return the same time (milliseconds since the epoch) + * as is reported by `System.currentTimeMillis()` + */ + def getTimeMillis(): Long = System.currentTimeMillis() + + /** + * @param targetTime block until the current time is at least this value + * @return current system time when wait has completed + */ + def waitTillTime(targetTime: Long): Long = { + var currentTime = 0L + currentTime = System.currentTimeMillis() + + var waitTime = targetTime - currentTime + if (waitTime <= 0) { + return currentTime + } + + val pollTime = math.max(waitTime / 10.0, minPollTime).toLong + + while (true) { + currentTime = System.currentTimeMillis() + waitTime = targetTime - currentTime + if (waitTime <= 0) { + return currentTime + } + val sleepTime = math.min(waitTime, pollTime) + Thread.sleep(sleepTime) + } + -1 + } } 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 414bc49a57f8a..474f79fb756f6 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -32,7 +32,6 @@ import org.apache.spark.executor._ import org.apache.spark.scheduler._ import org.apache.spark.storage._ import org.apache.spark._ -import org.apache.hadoop.hdfs.web.JsonUtil /** * Serializes SparkListener events to/from JSON. This protocol provides strong backwards- @@ -90,6 +89,8 @@ private[spark] object JsonProtocol { executorAddedToJson(executorAdded) case executorRemoved: SparkListenerExecutorRemoved => executorRemovedToJson(executorRemoved) + case logStart: SparkListenerLogStart => + logStartToJson(logStart) // These aren't used, but keeps compiler happy case SparkListenerExecutorMetricsUpdate(_, _) => JNothing } @@ -215,6 +216,11 @@ private[spark] object JsonProtocol { ("Removed Reason" -> executorRemoved.reason) } + def logStartToJson(logStart: SparkListenerLogStart): JValue = { + ("Event" -> Utils.getFormattedClassName(logStart)) ~ + ("Spark Version" -> SPARK_VERSION) + } + /** ------------------------------------------------------------------- * * JSON serialization methods for classes SparkListenerEvents depend on | * -------------------------------------------------------------------- */ @@ -294,22 +300,27 @@ private[spark] object JsonProtocol { ("Remote Blocks Fetched" -> shuffleReadMetrics.remoteBlocksFetched) ~ ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ - ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead) + ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead) ~ + ("Local Bytes Read" -> shuffleReadMetrics.localBytesRead) ~ + ("Total Records Read" -> shuffleReadMetrics.recordsRead) } def shuffleWriteMetricsToJson(shuffleWriteMetrics: ShuffleWriteMetrics): JValue = { ("Shuffle Bytes Written" -> shuffleWriteMetrics.shuffleBytesWritten) ~ - ("Shuffle Write Time" -> shuffleWriteMetrics.shuffleWriteTime) + ("Shuffle Write Time" -> shuffleWriteMetrics.shuffleWriteTime) ~ + ("Shuffle Records Written" -> shuffleWriteMetrics.shuffleRecordsWritten) } def inputMetricsToJson(inputMetrics: InputMetrics): JValue = { ("Data Read Method" -> inputMetrics.readMethod.toString) ~ - ("Bytes Read" -> inputMetrics.bytesRead) + ("Bytes Read" -> inputMetrics.bytesRead) ~ + ("Records Read" -> inputMetrics.recordsRead) } def outputMetricsToJson(outputMetrics: OutputMetrics): JValue = { ("Data Write Method" -> outputMetrics.writeMethod.toString) ~ - ("Bytes Written" -> outputMetrics.bytesWritten) + ("Bytes Written" -> outputMetrics.bytesWritten) ~ + ("Records Written" -> outputMetrics.recordsWritten) } def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { @@ -384,7 +395,8 @@ private[spark] object JsonProtocol { def executorInfoToJson(executorInfo: ExecutorInfo): JValue = { ("Host" -> executorInfo.executorHost) ~ - ("Total Cores" -> executorInfo.totalCores) + ("Total Cores" -> executorInfo.totalCores) ~ + ("Log Urls" -> mapToJson(executorInfo.logUrlMap)) } /** ------------------------------ * @@ -442,6 +454,7 @@ private[spark] object JsonProtocol { val applicationEnd = Utils.getFormattedClassName(SparkListenerApplicationEnd) val executorAdded = Utils.getFormattedClassName(SparkListenerExecutorAdded) val executorRemoved = Utils.getFormattedClassName(SparkListenerExecutorRemoved) + val logStart = Utils.getFormattedClassName(SparkListenerLogStart) (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) @@ -459,6 +472,7 @@ private[spark] object JsonProtocol { case `applicationEnd` => applicationEndFromJson(json) case `executorAdded` => executorAddedFromJson(json) case `executorRemoved` => executorRemovedFromJson(json) + case `logStart` => logStartFromJson(json) } } @@ -569,6 +583,11 @@ private[spark] object JsonProtocol { SparkListenerExecutorRemoved(time, executorId, reason) } + def logStartFromJson(json: JValue): SparkListenerLogStart = { + val sparkVersion = (json \ "Spark Version").extract[String] + SparkListenerLogStart(sparkVersion) + } + /** --------------------------------------------------------------------- * * JSON deserialization methods for classes SparkListenerEvents depend on | * ---------------------------------------------------------------------- */ @@ -670,6 +689,8 @@ private[spark] object JsonProtocol { metrics.incLocalBlocksFetched((json \ "Local Blocks Fetched").extract[Int]) metrics.incFetchWaitTime((json \ "Fetch Wait Time").extract[Long]) metrics.incRemoteBytesRead((json \ "Remote Bytes Read").extract[Long]) + metrics.incLocalBytesRead((json \ "Local Bytes Read").extractOpt[Long].getOrElse(0)) + metrics.incRecordsRead((json \ "Total Records Read").extractOpt[Long].getOrElse(0)) metrics } @@ -677,13 +698,16 @@ private[spark] object JsonProtocol { val metrics = new ShuffleWriteMetrics metrics.incShuffleBytesWritten((json \ "Shuffle Bytes Written").extract[Long]) metrics.incShuffleWriteTime((json \ "Shuffle Write Time").extract[Long]) + metrics.setShuffleRecordsWritten((json \ "Shuffle Records Written") + .extractOpt[Long].getOrElse(0)) metrics } def inputMetricsFromJson(json: JValue): InputMetrics = { val metrics = new InputMetrics( DataReadMethod.withName((json \ "Data Read Method").extract[String])) - metrics.addBytesRead((json \ "Bytes Read").extract[Long]) + metrics.incBytesRead((json \ "Bytes Read").extract[Long]) + metrics.incRecordsRead((json \ "Records Read").extractOpt[Long].getOrElse(0)) metrics } @@ -691,6 +715,7 @@ private[spark] object JsonProtocol { val metrics = new OutputMetrics( DataWriteMethod.withName((json \ "Data Write Method").extract[String])) metrics.setBytesWritten((json \ "Bytes Written").extract[Long]) + metrics.setRecordsWritten((json \ "Records Written").extractOpt[Long].getOrElse(0)) metrics } @@ -793,7 +818,8 @@ private[spark] object JsonProtocol { def executorInfoFromJson(json: JValue): ExecutorInfo = { val executorHost = (json \ "Host").extract[String] val totalCores = (json \ "Total Cores").extract[Int] - new ExecutorInfo(executorHost, totalCores) + val logUrls = mapFromJson(json \ "Log Urls").toMap + new ExecutorInfo(executorHost, totalCores, logUrls) } /** -------------------------------- * diff --git a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala index bd0aa4dc4650f..d60b8b9a31a9b 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -28,7 +28,8 @@ import org.apache.spark.Logging */ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { - private val listeners = new CopyOnWriteArrayList[L] + // Marked `private[spark]` for access in tests. + private[spark] val listeners = new CopyOnWriteArrayList[L] /** * Add a listener to listen events. This method is thread-safe and can be called in any thread. diff --git a/core/src/main/scala/org/apache/spark/util/ManualClock.scala b/core/src/main/scala/org/apache/spark/util/ManualClock.scala new file mode 100644 index 0000000000000..cf89c1782fd67 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/ManualClock.scala @@ -0,0 +1,69 @@ +/* + * 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 + +/** + * A `Clock` whose time can be manually set and modified. Its reported time does not change + * as time elapses, but only as its time is modified by callers. This is mainly useful for + * testing. + * + * @param time initial time (in milliseconds since the epoch) + */ +private[spark] class ManualClock(private var time: Long) extends Clock { + + /** + * @return `ManualClock` with initial time 0 + */ + def this() = this(0L) + + def getTimeMillis(): Long = + synchronized { + time + } + + /** + * @param timeToSet new time (in milliseconds) that the clock should represent + */ + def setTime(timeToSet: Long) = + synchronized { + time = timeToSet + notifyAll() + } + + /** + * @param timeToAdd time (in milliseconds) to add to the clock's time + */ + def advance(timeToAdd: Long) = + synchronized { + time += timeToAdd + notifyAll() + } + + /** + * @param targetTime block until the clock time is set or advanced to at least this time + * @return current time reported by the clock when waiting finishes + */ + def waitTillTime(targetTime: Long): Long = + synchronized { + while (time < targetTime) { + wait(100) + } + getTimeMillis() + } + +} diff --git a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala new file mode 100644 index 0000000000000..1e0ba5c28754a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.net.{URLClassLoader, URL} +import java.util.Enumeration +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConversions._ + +/** + * URL class loader that exposes the `addURL` and `getURLs` methods in URLClassLoader. + */ +private[spark] class MutableURLClassLoader(urls: Array[URL], parent: ClassLoader) + extends URLClassLoader(urls, parent) { + + override def addURL(url: URL): Unit = { + super.addURL(url) + } + + override def getURLs(): Array[URL] = { + super.getURLs() + } + +} + +/** + * A mutable class loader that gives preference to its own URLs over the parent class loader + * when loading classes and resources. + */ +private[spark] class ChildFirstURLClassLoader(urls: Array[URL], parent: ClassLoader) + extends MutableURLClassLoader(urls, null) { + + private val parentClassLoader = new ParentClassLoader(parent) + + /** + * Used to implement fine-grained class loading locks similar to what is done by Java 7. This + * prevents deadlock issues when using non-hierarchical class loaders. + * + * Note that due to Java 6 compatibility (and some issues with implementing class loaders in + * Scala), Java 7's `ClassLoader.registerAsParallelCapable` method is not called. + */ + private val locks = new ConcurrentHashMap[String, Object]() + + override def loadClass(name: String, resolve: Boolean): Class[_] = { + var lock = locks.get(name) + if (lock == null) { + val newLock = new Object() + lock = locks.putIfAbsent(name, newLock) + if (lock == null) { + lock = newLock + } + } + + lock.synchronized { + try { + super.loadClass(name, resolve) + } catch { + case e: ClassNotFoundException => + parentClassLoader.loadClass(name, resolve) + } + } + } + + override def getResource(name: String): URL = { + val url = super.findResource(name) + val res = if (url != null) url else parentClassLoader.getResource(name) + res + } + + override def getResources(name: String): Enumeration[URL] = { + val urls = super.findResources(name) + val res = + if (urls != null && urls.hasMoreElements()) { + urls + } else { + parentClassLoader.getResources(name) + } + res + } + + override def addURL(url: URL) { + super.addURL(url) + } + +} diff --git a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala index 3abc12681fe9a..6d8d9e8da3678 100644 --- a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala @@ -18,7 +18,7 @@ package org.apache.spark.util /** - * A class loader which makes findClass accesible to the child + * A class loader which makes some protected methods in ClassLoader accesible. */ private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) { @@ -29,4 +29,9 @@ private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader( override def loadClass(name: String): Class[_] = { super.loadClass(name) } + + override def loadClass(name: String, resolve: Boolean): Class[_] = { + super.loadClass(name, resolve) + } + } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 86ac307fc84ba..d3dc1d09cb7b4 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -21,8 +21,9 @@ import java.io._ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer -import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor} -import java.util.{Locale, Properties, Random, UUID} +import java.util.{Properties, Locale, Random, UUID} +import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} +import javax.net.ssl.HttpsURLConnection import scala.collection.JavaConversions._ import scala.collection.Map @@ -37,6 +38,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} +import org.apache.hadoop.security.UserGroupInformation import org.apache.log4j.PropertyConfigurator import org.eclipse.jetty.util.MultiException import org.json4s._ @@ -60,6 +62,9 @@ private[spark] object CallSite { private[spark] object Utils extends Logging { val random = new Random() + private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 + @volatile private var localRootDirs: Array[String] = null + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -209,8 +214,8 @@ private[spark] object Utils extends Logging { // Is the path already registered to be deleted via a shutdown hook ? def hasShutdownDeleteTachyonDir(file: TachyonFile): Boolean = { val absolutePath = file.getPath() - shutdownDeletePaths.synchronized { - shutdownDeletePaths.contains(absolutePath) + shutdownDeleteTachyonPaths.synchronized { + shutdownDeleteTachyonPaths.contains(absolutePath) } } @@ -246,13 +251,28 @@ private[spark] object Utils extends Logging { retval } + /** + * JDK equivalent of `chmod 700 file`. + * + * @param file the file whose permissions will be modified + * @return true if the permissions were successfully changed, false otherwise. + */ + def chmod700(file: File): Boolean = { + file.setReadable(false, false) && + file.setReadable(true, true) && + file.setWritable(false, false) && + file.setWritable(true, true) && + file.setExecutable(false, false) && + file.setExecutable(true, true) + } + /** * Create a directory inside the given parent directory. The directory is guaranteed to be * newly created, and is not marked for automatic deletion. */ - def createDirectory(root: String): File = { + def createDirectory(root: String, namePrefix: String = "spark"): File = { var attempts = 0 - val maxAttempts = 10 + val maxAttempts = MAX_DIR_CREATION_ATTEMPTS var dir: File = null while (dir == null) { attempts += 1 @@ -261,7 +281,7 @@ private[spark] object Utils extends Logging { maxAttempts + " attempts!") } try { - dir = new File(root, "spark-" + UUID.randomUUID.toString) + dir = new File(root, namePrefix + "-" + UUID.randomUUID.toString) if (dir.exists() || !dir.mkdirs()) { dir = null } @@ -275,8 +295,10 @@ private[spark] object Utils extends Logging { * Create a temporary directory inside the given parent directory. The directory will be * automatically deleted when the VM shuts down. */ - def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { - val dir = createDirectory(root) + def createTempDir( + root: String = System.getProperty("java.io.tmpdir"), + namePrefix: String = "spark"): File = { + val dir = createDirectory(root, namePrefix) registerShutdownDeleteDir(dir) dir } @@ -359,8 +381,10 @@ private[spark] object Utils extends Logging { } /** - * Download a file to target directory. Supports fetching the file in a variety of ways, - * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. + * Download a file or directory to target directory. Supports fetching the file in a variety of + * ways, including HTTP, Hadoop-compatible filesystems, and files on a standard filesystem, based + * on the URL parameter. Fetching directories is only supported from Hadoop-compatible + * filesystems. * * If `useCache` is true, first attempts to fetch the file to a local cache that's shared * across executors running the same application. `useCache` is used mainly for @@ -417,6 +441,12 @@ private[spark] object Utils extends Logging { } // Make the file executable - That's necessary for scripts FileUtil.chmod(targetFile.getAbsolutePath, "a+x") + + // Windows does not grant read permission by default to non-admin users + // Add read permission to owner explicitly + if (isWindows) { + FileUtil.chmod(targetFile.getAbsolutePath, "u+r") + } } /** @@ -429,7 +459,6 @@ private[spark] object Utils extends Logging { * * @param url URL that `sourceFile` originated from, for logging purposes. * @param in InputStream to download. - * @param tempFile File path to download `in` to. * @param destFile File path to move `tempFile` to. * @param fileOverwrite Whether to delete/overwrite an existing `destFile` that does not match * `sourceFile` @@ -437,9 +466,11 @@ private[spark] object Utils extends Logging { private def downloadFile( url: String, in: InputStream, - tempFile: File, destFile: File, fileOverwrite: Boolean): Unit = { + val tempFile = File.createTempFile("fetchFileTemp", null, + new File(destFile.getParentFile.getAbsolutePath)) + logInfo(s"Fetching $url to $tempFile") try { val out = new FileOutputStream(tempFile) @@ -478,7 +509,7 @@ private[spark] object Utils extends Logging { removeSourceFile: Boolean = false): Unit = { if (destFile.exists) { - if (!Files.equal(sourceFile, destFile)) { + if (!filesEqualRecursive(sourceFile, destFile)) { if (fileOverwrite) { logInfo( s"File $destFile exists and does not match contents of $url, replacing it with $url" @@ -513,13 +544,44 @@ private[spark] object Utils extends Logging { Files.move(sourceFile, destFile) } else { logInfo(s"Copying ${sourceFile.getAbsolutePath} to ${destFile.getAbsolutePath}") - Files.copy(sourceFile, destFile) + copyRecursive(sourceFile, destFile) + } + } + + private def filesEqualRecursive(file1: File, file2: File): Boolean = { + if (file1.isDirectory && file2.isDirectory) { + val subfiles1 = file1.listFiles() + val subfiles2 = file2.listFiles() + if (subfiles1.size != subfiles2.size) { + return false + } + subfiles1.sortBy(_.getName).zip(subfiles2.sortBy(_.getName)).forall { + case (f1, f2) => filesEqualRecursive(f1, f2) + } + } else if (file1.isFile && file2.isFile) { + Files.equal(file1, file2) + } else { + false + } + } + + private def copyRecursive(source: File, dest: File): Unit = { + if (source.isDirectory) { + if (!dest.mkdir()) { + throw new IOException(s"Failed to create directory ${dest.getPath}") + } + val subfiles = source.listFiles() + subfiles.foreach(f => copyRecursive(f, new File(dest, f.getName))) + } else { + Files.copy(source, dest) } } /** - * Download a file to target directory. Supports fetching the file in a variety of ways, - * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. + * Download a file or directory to target directory. Supports fetching the file in a variety of + * ways, including HTTP, Hadoop-compatible filesystems, and files on a standard filesystem, based + * on the URL parameter. Fetching directories is only supported from Hadoop-compatible + * filesystems. * * Throws SparkException if the target file already exists and has different contents than * the requested file. @@ -531,14 +593,11 @@ private[spark] object Utils extends Logging { conf: SparkConf, securityMgr: SecurityManager, hadoopConf: Configuration) { - val tempFile = File.createTempFile("fetchFileTemp", null, new File(targetDir.getAbsolutePath)) val targetFile = new File(targetDir, filename) val uri = new URI(url) val fileOverwrite = conf.getBoolean("spark.files.overwrite", defaultValue = false) Option(uri.getScheme).getOrElse("file") match { case "http" | "https" | "ftp" => - logInfo("Fetching " + url + " to " + tempFile) - var uc: URLConnection = null if (securityMgr.isAuthenticationEnabled()) { logDebug("fetchFile with security enabled") @@ -549,23 +608,55 @@ private[spark] object Utils extends Logging { logDebug("fetchFile not using security") uc = new URL(url).openConnection() } + Utils.setupSecureURLConnection(uc, securityMgr) val timeout = conf.getInt("spark.files.fetchTimeout", 60) * 1000 uc.setConnectTimeout(timeout) uc.setReadTimeout(timeout) uc.connect() val in = uc.getInputStream() - downloadFile(url, in, tempFile, targetFile, fileOverwrite) + downloadFile(url, in, targetFile, fileOverwrite) case "file" => // In the case of a local file, copy the local file to the target directory. // Note the difference between uri vs url. val sourceFile = if (uri.isAbsolute) new File(uri) else new File(url) copyFile(url, sourceFile, targetFile, fileOverwrite) case _ => - // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others val fs = getHadoopFileSystem(uri, hadoopConf) - val in = fs.open(new Path(uri)) - downloadFile(url, in, tempFile, targetFile, fileOverwrite) + val path = new Path(uri) + fetchHcfsFile(path, targetDir, fs, conf, hadoopConf, fileOverwrite, + filename = Some(filename)) + } + } + + /** + * Fetch a file or directory from a Hadoop-compatible filesystem. + * + * Visible for testing + */ + private[spark] def fetchHcfsFile( + path: Path, + targetDir: File, + fs: FileSystem, + conf: SparkConf, + hadoopConf: Configuration, + fileOverwrite: Boolean, + filename: Option[String] = None): Unit = { + if (!targetDir.exists() && !targetDir.mkdir()) { + throw new IOException(s"Failed to create directory ${targetDir.getPath}") + } + val dest = new File(targetDir, filename.getOrElse(path.getName)) + if (fs.isFile(path)) { + val in = fs.open(path) + try { + downloadFile(path.toString, in, dest, fileOverwrite) + } finally { + in.close() + } + } else { + fs.listStatus(path).foreach { fileStatus => + fetchHcfsFile(fileStatus.getPath(), dest, fs, conf, hadoopConf, fileOverwrite) + } } } @@ -597,28 +688,56 @@ private[spark] object Utils extends Logging { * and returns only the directories that exist / could be created. * * If no directories could be created, this will return an empty list. + * + * This method will cache the local directories for the application when it's first invoked. + * So calling it multiple times with a different configuration will always return the same + * set of directories. */ private[spark] def getOrCreateLocalRootDirs(conf: SparkConf): Array[String] = { - val confValue = if (isRunningInYarnContainer(conf)) { + if (localRootDirs == null) { + this.synchronized { + if (localRootDirs == null) { + localRootDirs = getOrCreateLocalRootDirsImpl(conf) + } + } + } + localRootDirs + } + + private def getOrCreateLocalRootDirsImpl(conf: SparkConf): Array[String] = { + if (isRunningInYarnContainer(conf)) { // If we are in yarn mode, systems can have different disk layouts so we must set it - // to what Yarn on this system said was available. - getYarnLocalDirs(conf) + // to what Yarn on this system said was available. Note this assumes that Yarn has + // created the directories already, and that they are secured so that only the + // user has access to them. + getYarnLocalDirs(conf).split(",") + } else if (conf.getenv("SPARK_EXECUTOR_DIRS") != null) { + conf.getenv("SPARK_EXECUTOR_DIRS").split(File.pathSeparator) } else { - Option(conf.getenv("SPARK_LOCAL_DIRS")).getOrElse( - conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) - } - val rootDirs = confValue.split(',') - logDebug(s"Getting/creating local root dirs at '$confValue'") - - rootDirs.flatMap { rootDir => - val localDir: File = new File(rootDir) - val foundLocalDir = localDir.exists || localDir.mkdirs() - if (!foundLocalDir) { - logError(s"Failed to create local root dir in $rootDir. Ignoring this directory.") - None - } else { - Some(rootDir) - } + // In non-Yarn mode (or for the driver in yarn-client mode), we cannot trust the user + // configuration to point to a secure directory. So create a subdirectory with restricted + // permissions under each listed directory. + Option(conf.getenv("SPARK_LOCAL_DIRS")) + .getOrElse(conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) + .split(",") + .flatMap { root => + try { + val rootDir = new File(root) + if (rootDir.exists || rootDir.mkdirs()) { + val dir = createTempDir(root) + chmod700(dir) + Some(dir.getAbsolutePath) + } else { + logError(s"Failed to create dir in $root. Ignoring this directory.") + None + } + } catch { + case e: IOException => + logError(s"Failed to create local root dir in $root. Ignoring this directory.") + None + } + } + .toArray } } @@ -637,6 +756,11 @@ private[spark] object Utils extends Logging { localDirs } + /** Used by unit tests. Do not call from other places. */ + private[spark] def clearLocalRootDirs(): Unit = { + localRootDirs = null + } + /** * Shuffle the elements of a collection into a random order, returning the * result in a new collection. Unlike scala.util.Random.shuffle, this method @@ -1067,9 +1191,9 @@ private[spark] object Utils extends Logging { // finding the call site of a method. val SPARK_CORE_CLASS_REGEX = """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?(\.broadcast)?\.[A-Z]""".r - val SCALA_CLASS_REGEX = """^scala""".r + val SCALA_CORE_CLASS_PREFIX = "scala" val isSparkCoreClass = SPARK_CORE_CLASS_REGEX.findFirstIn(className).isDefined - val isScalaClass = SCALA_CLASS_REGEX.findFirstIn(className).isDefined + val isScalaClass = className.startsWith(SCALA_CORE_CLASS_PREFIX) // If the class is a Spark internal class or a Scala class, then exclude. isSparkCoreClass || isScalaClass } @@ -1312,9 +1436,14 @@ private[spark] object Utils extends Logging { hashAbs } - /** Returns a copy of the system properties that is thread-safe to iterator over. */ - def getSystemProperties(): Map[String, String] = { - System.getProperties.clone().asInstanceOf[java.util.Properties].toMap[String, String] + /** Returns the system properties map that is thread-safe to iterator over. It gets the + * properties which have been set explicitly, as well as those for which only a default value + * has been defined. */ + def getSystemProperties: Map[String, String] = { + val sysProps = for (key <- System.getProperties.stringPropertyNames()) yield + (key, System.getProperty(key)) + + sysProps.toMap } /** @@ -1780,6 +1909,20 @@ private[spark] object Utils extends Logging { PropertyConfigurator.configure(pro) } + /** + * If the given URL connection is HttpsURLConnection, it sets the SSL socket factory and + * the host verifier from the given security manager. + */ + def setupSecureURLConnection(urlConnection: URLConnection, sm: SecurityManager): URLConnection = { + urlConnection match { + case https: HttpsURLConnection => + sm.sslSocketFactory.foreach(https.setSSLSocketFactory) + sm.hostnameVerifier.foreach(https.setHostnameVerifier) + https + case connection => connection + } + } + def invoke( clazz: Class[_], obj: AnyRef, @@ -1872,6 +2015,16 @@ private[spark] object Utils extends Logging { throw new SparkException("Invalid master URL: " + sparkUrl, e) } } + + /** + * Returns the current user name. This is the currently logged in user, unless that's been + * overridden by the `SPARK_USER` environment variable. + */ + def getCurrentUserName(): String = { + Option(System.getenv("SPARK_USER")) + .getOrElse(UserGroupInformation.getCurrentUser().getUserName()) + } + } /** diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 6ba03841f746b..d69f2d9048055 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -723,6 +723,7 @@ private[spark] class ExternalSorter[K, V, C]( partitionWriters.foreach(_.commitAndClose()) var out: FileOutputStream = null var in: FileInputStream = null + val writeStartTime = System.nanoTime try { out = new FileOutputStream(outputFile, true) for (i <- 0 until numPartitions) { @@ -739,6 +740,8 @@ private[spark] class ExternalSorter[K, V, C]( if (in != null) { in.close() } + context.taskMetrics.shuffleWriteMetrics.foreach( + _.incShuffleWriteTime(System.nanoTime - writeStartTime)) } } else { // Either we're not bypassing merge-sort or we have only in-memory data; get an iterator by @@ -763,6 +766,7 @@ private[spark] class ExternalSorter[K, V, C]( if (curWriteMetrics != null) { m.incShuffleBytesWritten(curWriteMetrics.shuffleBytesWritten) m.incShuffleWriteTime(curWriteMetrics.shuffleWriteTime) + m.incShuffleRecordsWritten(curWriteMetrics.shuffleRecordsWritten) } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala index 7e76d060d6000..b6c380a8eea9f 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala @@ -71,12 +71,21 @@ class PrimitiveVector[@specialized(Long, Int, Double) V: ClassTag](initialSize: /** Resizes the array, dropping elements if the total length decreases. */ def resize(newLength: Int): PrimitiveVector[V] = { - val newArray = new Array[V](newLength) - _array.copyToArray(newArray) - _array = newArray + _array = copyArrayWithLength(newLength) if (newLength < _numElements) { _numElements = newLength } this } + + /** Return a trimmed version of the underlying array. */ + def toArray: Array[V] = { + copyArrayWithLength(size) + } + + private def copyArrayWithLength(length: Int): Array[V] = { + val copy = new Array[V](length) + _array.copyToArray(copy) + copy + } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala index 65a7b4e0d497b..dfcfb66af8613 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala @@ -36,11 +36,4 @@ private[spark] class SizeTrackingVector[T: ClassTag] resetSamples() this } - - /** - * Return a trimmed version of the underlying array. - */ - def toArray: Array[T] = { - super.iterator.toArray - } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala index 9f54312074856..747ecf075a397 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -42,9 +42,6 @@ private[spark] trait Spillable[C] extends Logging { // Memory manager that can be used to acquire/release memory private[this] val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager - // Threshold for `elementsRead` before we start tracking this collection's memory usage - private[this] val trackMemoryThreshold = 1000 - // Initial threshold for the size of a collection before we start tracking its memory usage // Exposed for testing private[this] val initialMemoryThreshold: Long = @@ -72,8 +69,7 @@ private[spark] trait Spillable[C] extends Logging { * @return true if `collection` was spilled to disk; false otherwise */ protected def maybeSpill(collection: C, currentMemory: Long): Boolean = { - if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 && - currentMemory >= myMemoryThreshold) { + if (elementsRead % 32 == 0 && currentMemory >= myMemoryThreshold) { // Claim up to double our current memory from the shuffle memory pool val amountToRequest = 2 * currentMemory - myMemoryThreshold val granted = shuffleMemoryManager.tryToAcquire(amountToRequest) diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index b16a1e9460286..74e88c767ee07 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -708,6 +708,10 @@ public void javaDoubleRDDHistoGram() { // Test with provided buckets long[] histogram = rdd.histogram(expected_buckets); Assert.assertArrayEquals(expected_counts, histogram); + // SPARK-5744 + Assert.assertArrayEquals( + new long[] {0}, + sc.parallelizeDoubles(new ArrayList(0), 1).histogram(new double[]{0.0, 1.0})); } @Test diff --git a/core/src/test/java/org/apache/spark/util/collection/TestTimSort.java b/core/src/test/java/org/apache/spark/util/collection/TestTimSort.java new file mode 100644 index 0000000000000..45772b6d3c20d --- /dev/null +++ b/core/src/test/java/org/apache/spark/util/collection/TestTimSort.java @@ -0,0 +1,134 @@ +/** + * Copyright 2015 Stijn de Gouw + * + * 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. + * + */ +package org.apache.spark.util.collection; + +import java.util.*; + +/** + * This codes generates a int array which fails the standard TimSort. + * + * The blog that reported the bug + * http://www.envisage-project.eu/timsort-specification-and-verification/ + * + * This codes was originally wrote by Stijn de Gouw, modified by Evan Yu to adapt to + * our test suite. + * + * https://github.com/abstools/java-timsort-bug + * https://github.com/abstools/java-timsort-bug/blob/master/LICENSE + */ +public class TestTimSort { + + private static final int MIN_MERGE = 32; + + /** + * Returns an array of integers that demonstrate the bug in TimSort + */ + public static int[] getTimSortBugTestSet(int length) { + int minRun = minRunLength(length); + List runs = runsJDKWorstCase(minRun, length); + return createArray(runs, length); + } + + private static int minRunLength(int n) { + int r = 0; // Becomes 1 if any 1 bits are shifted off + while (n >= MIN_MERGE) { + r |= (n & 1); + n >>= 1; + } + return n + r; + } + + private static int[] createArray(List runs, int length) { + int[] a = new int[length]; + Arrays.fill(a, 0); + int endRun = -1; + for (long len : runs) { + a[endRun += len] = 1; + } + a[length - 1] = 0; + return a; + } + + /** + * Fills runs
with a sequence of run lengths of the form
+ * Y_n x_{n,1} x_{n,2} ... x_{n,l_n}
+ * Y_{n-1} x_{n-1,1} x_{n-1,2} ... x_{n-1,l_{n-1}}
+ * ...
+ * Y_1 x_{1,1} x_{1,2} ... x_{1,l_1}
+ * The Y_i's are chosen to satisfy the invariant throughout execution, + * but the x_{i,j}'s are merged (byTimSort.mergeCollapse
) + * into an X_i that violates the invariant. + * + * @param length The sum of all run lengths that will be added toruns
. + */ + private static ListrunsJDKWorstCase(int minRun, int length) { + List runs = new ArrayList (); + + long runningTotal = 0, Y = minRun + 4, X = minRun; + + while (runningTotal + Y + X <= length) { + runningTotal += X + Y; + generateJDKWrongElem(runs, minRun, X); + runs.add(0, Y); + // X_{i+1} = Y_i + x_{i,1} + 1, since runs.get(1) = x_{i,1} + X = Y + runs.get(1) + 1; + // Y_{i+1} = X_{i+1} + Y_i + 1 + Y += X + 1; + } + + if (runningTotal + X <= length) { + runningTotal += X; + generateJDKWrongElem(runs, minRun, X); + } + + runs.add(length - runningTotal); + return runs; + } + + /** + * Adds a sequence x_1, ..., x_n of run lengths to runs
such that:
+ * 1. X = x_1 + ... + x_n
+ * 2. x_j >= minRun for all j
+ * 3. x_1 + ... + x_{j-2} < x_j < x_1 + ... + x_{j-1} for all j
+ * These conditions guarantee that TimSort merges all x_j's one by one + * (resulting in X) using only merges on the second-to-last element. + * + * @param X The sum of the sequence that should be added to runs. + */ + private static void generateJDKWrongElem(Listruns, int minRun, long X) { + for (long newTotal; X >= 2 * minRun + 1; X = newTotal) { + //Default strategy + newTotal = X / 2 + 1; + //Specialized strategies + if (3 * minRun + 3 <= X && X <= 4 * minRun + 1) { + // add x_1=MIN+1, x_2=MIN, x_3=X-newTotal to runs + newTotal = 2 * minRun + 1; + } else if (5 * minRun + 5 <= X && X <= 6 * minRun + 5) { + // add x_1=MIN+1, x_2=MIN, x_3=MIN+2, x_4=X-newTotal to runs + newTotal = 3 * minRun + 3; + } else if (8 * minRun + 9 <= X && X <= 10 * minRun + 9) { + // add x_1=MIN+1, x_2=MIN, x_3=MIN+2, x_4=2MIN+2, x_5=X-newTotal to runs + newTotal = 5 * minRun + 5; + } else if (13 * minRun + 15 <= X && X <= 16 * minRun + 17) { + // add x_1=MIN+1, x_2=MIN, x_3=MIN+2, x_4=2MIN+2, x_5=3MIN+4, x_6=X-newTotal to runs + newTotal = 8 * minRun + 9; + } + runs.add(0, X - newTotal); + } + runs.add(0, X); + } +} diff --git a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java b/core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java similarity index 93% rename from core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java rename to core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java index e9ec700e32e15..e38bc38949d7c 100644 --- a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java +++ b/core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.spark.util; +package test.org.apache.spark; import org.apache.spark.TaskContext; +import org.apache.spark.util.TaskCompletionListener; /** diff --git a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java new file mode 100644 index 0000000000000..4a918f725dc91 --- /dev/null +++ b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark; + +import org.apache.spark.TaskContext; + +/** + * Something to make sure that TaskContext can be used in Java. + */ +public class JavaTaskContextCompileCheck { + + public static void test() { + TaskContext tc = TaskContext.get(); + + tc.isCompleted(); + tc.isInterrupted(); + tc.isRunningLocally(); + + tc.addTaskCompletionListener(new JavaTaskCompletionListenerImpl()); + + tc.attemptNumber(); + tc.partitionId(); + tc.stageId(); + tc.taskAttemptId(); + } +} diff --git a/core/src/test/resources/keystore b/core/src/test/resources/keystore new file mode 100644 index 0000000000000..f8310e39ba1e0 Binary files /dev/null and b/core/src/test/resources/keystore differ diff --git a/core/src/test/resources/test_metrics_system.properties b/core/src/test/resources/test_metrics_system.properties index 35d0bd3b8d0b8..4e8b8465696e5 100644 --- a/core/src/test/resources/test_metrics_system.properties +++ b/core/src/test/resources/test_metrics_system.properties @@ -18,7 +18,5 @@ *.sink.console.period = 10 *.sink.console.unit = seconds test.sink.console.class = org.apache.spark.metrics.sink.ConsoleSink -test.sink.dummy.class = org.apache.spark.metrics.sink.DummySink -test.source.dummy.class = org.apache.spark.metrics.source.DummySource test.sink.console.period = 20 test.sink.console.unit = minutes diff --git a/core/src/test/resources/truststore b/core/src/test/resources/truststore new file mode 100644 index 0000000000000..a6b1d46e1f391 Binary files /dev/null and b/core/src/test/resources/truststore differ diff --git a/core/src/test/resources/untrusted-keystore b/core/src/test/resources/untrusted-keystore new file mode 100644 index 0000000000000..6015b02caa128 Binary files /dev/null and b/core/src/test/resources/untrusted-keystore differ diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index f087fc550dde3..bd0f8bdefa171 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark import scala.collection.mutable +import scala.ref.WeakReference import org.scalatest.FunSuite import org.scalatest.Matchers @@ -136,4 +137,23 @@ class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { } } + test ("garbage collection") { + // Create an accumulator and let it go out of scope to test that it's properly garbage collected + sc = new SparkContext("local", "test") + var acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) + val accId = acc.id + val ref = WeakReference(acc) + + // Ensure the accumulator is present + assert(ref.get.isDefined) + + // Remove the explicit reference to it and allow weak reference to get garbage collected + acc = null + System.gc() + assert(ref.get.isEmpty) + + Accumulators.remove(accId) + assert(!Accumulators.originals.get(accId).isDefined) + } + } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index d7d9dc7b50f30..4b25c200a695a 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -17,16 +17,18 @@ package org.apache.spark +import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfter, FunSuite} -import org.scalatest.mock.EasyMockSugar +import org.scalatest.mock.MockitoSugar -import org.apache.spark.executor.{DataReadMethod, TaskMetrics} +import org.apache.spark.executor.DataReadMethod import org.apache.spark.rdd.RDD import org.apache.spark.storage._ // TODO: Test the CacheManager's thread-safety aspects -class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar { - var sc : SparkContext = _ +class CacheManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAfter + with MockitoSugar { + var blockManager: BlockManager = _ var cacheManager: CacheManager = _ var split: Partition = _ @@ -57,10 +59,6 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar }.cache() } - after { - sc.stop() - } - test("get uncached rdd") { // Do not mock this test, because attempting to match Array[Any], which is not covariant, // in blockManager.put is a losing battle. You have been warned. @@ -75,29 +73,21 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } test("get cached rdd") { - expecting { - val result = new BlockResult(Array(5, 6, 7).iterator, DataReadMethod.Memory, 12) - blockManager.get(RDDBlockId(0, 0)).andReturn(Some(result)) - } + val result = new BlockResult(Array(5, 6, 7).iterator, DataReadMethod.Memory, 12) + when(blockManager.get(RDDBlockId(0, 0))).thenReturn(Some(result)) - whenExecuting(blockManager) { - val context = new TaskContextImpl(0, 0, 0, 0) - val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) - assert(value.toList === List(5, 6, 7)) - } + val context = new TaskContextImpl(0, 0, 0, 0) + val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) + assert(value.toList === List(5, 6, 7)) } test("get uncached local rdd") { - expecting { - // Local computation should not persist the resulting value, so don't expect a put(). - blockManager.get(RDDBlockId(0, 0)).andReturn(None) - } + // Local computation should not persist the resulting value, so don't expect a put(). + when(blockManager.get(RDDBlockId(0, 0))).thenReturn(None) - whenExecuting(blockManager) { - val context = new TaskContextImpl(0, 0, 0, 0, true) - val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) - assert(value.toList === List(1, 2, 3, 4)) - } + val context = new TaskContextImpl(0, 0, 0, 0, true) + val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) + assert(value.toList === List(1, 2, 3, 4)) } test("verify task metrics updated correctly") { diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index ae2ae7ed0d3aa..cdfaacee7da40 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -382,6 +382,10 @@ class CleanerTester( toBeCleanedBroadcstIds -= broadcastId logInfo("Broadcast" + broadcastId + " cleaned") } + + def accumCleaned(accId: Long): Unit = { + logInfo("Cleaned accId " + accId + " cleaned") + } } val MAX_VALIDATION_ATTEMPTS = 10 diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 0e4df17c1bf87..abfcee75728dc 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -22,7 +22,8 @@ import scala.collection.mutable import org.scalatest.{FunSuite, PrivateMethodTester} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ -import org.apache.spark.storage.BlockManagerId +import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.util.ManualClock /** * Test add and remove behavior of ExecutorAllocationManager. @@ -32,24 +33,23 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { import ExecutorAllocationManagerSuite._ test("verify min/max executors") { - // No min or max val conf = new SparkConf() .setMaster("local") .setAppName("test-executor-allocation-manager") .set("spark.dynamicAllocation.enabled", "true") .set("spark.dynamicAllocation.testing", "true") - intercept[SparkException] { new SparkContext(conf) } - SparkEnv.get.stop() // cleanup the created environment - SparkContext.clearActiveContext() + val sc0 = new SparkContext(conf) + assert(sc0.executorAllocationManager.isDefined) + sc0.stop() - // Only min - val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1") + // Min < 0 + val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "-1") intercept[SparkException] { new SparkContext(conf1) } SparkEnv.get.stop() SparkContext.clearActiveContext() - // Only max - val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2") + // Max < 0 + val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "-1") intercept[SparkException] { new SparkContext(conf2) } SparkEnv.get.stop() SparkContext.clearActiveContext() @@ -145,8 +145,8 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { // Verify that running a task reduces the cap sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, 3))) - sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( - 0L, BlockManagerId("executor-1", "host1", 1), 100L)) + sc.listenerBus.postToAll(SparkListenerExecutorAdded( + 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) assert(numExecutorsPending(manager) === 4) assert(addExecutors(manager) === 1) @@ -176,6 +176,33 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(numExecutorsPending(manager) === 9) } + test("cancel pending executors when no longer needed") { + sc = createSparkContext(1, 10) + val manager = sc.executorAllocationManager.get + sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(2, 5))) + + assert(numExecutorsPending(manager) === 0) + assert(numExecutorsToAdd(manager) === 1) + assert(addExecutors(manager) === 1) + assert(numExecutorsPending(manager) === 1) + assert(numExecutorsToAdd(manager) === 2) + assert(addExecutors(manager) === 2) + assert(numExecutorsPending(manager) === 3) + + val task1Info = createTaskInfo(0, 0, "executor-1") + sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, task1Info)) + + assert(numExecutorsToAdd(manager) === 4) + assert(addExecutors(manager) === 2) + + val task2Info = createTaskInfo(1, 0, "executor-1") + sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, task2Info)) + sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, null, task1Info, null)) + sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, null, task2Info, null)) + + assert(adjustRequestedExecutors(manager) === -1) + } + test("remove executors") { sc = createSparkContext(5, 10) val manager = sc.executorAllocationManager.get @@ -271,15 +298,15 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeExecutor(manager, "5")) assert(removeExecutor(manager, "6")) assert(executorIds(manager).size === 10) - assert(addExecutors(manager) === 0) // still at upper limit + assert(addExecutors(manager) === 1) onExecutorRemoved(manager, "3") onExecutorRemoved(manager, "4") assert(executorIds(manager).size === 8) // Add succeeds again, now that we are no longer at the upper limit // Number of executors added restarts at 1 - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 1) // upper limit reached again + assert(addExecutors(manager) === 2) + assert(addExecutors(manager) === 1) // upper limit reached assert(addExecutors(manager) === 0) assert(executorIds(manager).size === 8) onExecutorRemoved(manager, "5") @@ -287,9 +314,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { onExecutorAdded(manager, "13") onExecutorAdded(manager, "14") assert(executorIds(manager).size === 8) - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 1) // upper limit reached again - assert(addExecutors(manager) === 0) + assert(addExecutors(manager) === 0) // still at upper limit onExecutorAdded(manager, "15") onExecutorAdded(manager, "16") assert(executorIds(manager).size === 10) @@ -297,7 +322,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { test("starting/canceling add timer") { sc = createSparkContext(2, 10) - val clock = new TestClock(8888L) + val clock = new ManualClock(8888L) val manager = sc.executorAllocationManager.get manager.setClock(clock) @@ -306,21 +331,21 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { onSchedulerBacklogged(manager) val firstAddTime = addTime(manager) assert(firstAddTime === clock.getTimeMillis + schedulerBacklogTimeout * 1000) - clock.tick(100L) + clock.advance(100L) onSchedulerBacklogged(manager) assert(addTime(manager) === firstAddTime) // timer is already started - clock.tick(200L) + clock.advance(200L) onSchedulerBacklogged(manager) assert(addTime(manager) === firstAddTime) onSchedulerQueueEmpty(manager) // Restart add timer - clock.tick(1000L) + clock.advance(1000L) assert(addTime(manager) === NOT_SET) onSchedulerBacklogged(manager) val secondAddTime = addTime(manager) assert(secondAddTime === clock.getTimeMillis + schedulerBacklogTimeout * 1000) - clock.tick(100L) + clock.advance(100L) onSchedulerBacklogged(manager) assert(addTime(manager) === secondAddTime) // timer is already started assert(addTime(manager) !== firstAddTime) @@ -329,7 +354,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { test("starting/canceling remove timers") { sc = createSparkContext(2, 10) - val clock = new TestClock(14444L) + val clock = new ManualClock(14444L) val manager = sc.executorAllocationManager.get manager.setClock(clock) @@ -342,17 +367,17 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).contains("1")) val firstRemoveTime = removeTimes(manager)("1") assert(firstRemoveTime === clock.getTimeMillis + executorIdleTimeout * 1000) - clock.tick(100L) + clock.advance(100L) onExecutorIdle(manager, "1") assert(removeTimes(manager)("1") === firstRemoveTime) // timer is already started - clock.tick(200L) + clock.advance(200L) onExecutorIdle(manager, "1") assert(removeTimes(manager)("1") === firstRemoveTime) - clock.tick(300L) + clock.advance(300L) onExecutorIdle(manager, "2") assert(removeTimes(manager)("2") !== firstRemoveTime) // different executor assert(removeTimes(manager)("2") === clock.getTimeMillis + executorIdleTimeout * 1000) - clock.tick(400L) + clock.advance(400L) onExecutorIdle(manager, "3") assert(removeTimes(manager)("3") !== firstRemoveTime) assert(removeTimes(manager)("3") === clock.getTimeMillis + executorIdleTimeout * 1000) @@ -361,7 +386,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).contains("3")) // Restart remove timer - clock.tick(1000L) + clock.advance(1000L) onExecutorBusy(manager, "1") assert(removeTimes(manager).size === 2) onExecutorIdle(manager, "1") @@ -377,7 +402,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { test("mock polling loop with no events") { sc = createSparkContext(1, 20) val manager = sc.executorAllocationManager.get - val clock = new TestClock(2020L) + val clock = new ManualClock(2020L) manager.setClock(clock) // No events - we should not be adding or removing @@ -386,15 +411,15 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { schedule(manager) assert(numExecutorsPending(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) - clock.tick(100L) + clock.advance(100L) schedule(manager) assert(numExecutorsPending(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) - clock.tick(1000L) + clock.advance(1000L) schedule(manager) assert(numExecutorsPending(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) - clock.tick(10000L) + clock.advance(10000L) schedule(manager) assert(numExecutorsPending(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) @@ -402,57 +427,57 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { test("mock polling loop add behavior") { sc = createSparkContext(1, 20) - val clock = new TestClock(2020L) + val clock = new ManualClock(2020L) val manager = sc.executorAllocationManager.get manager.setClock(clock) sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 1000))) // Scheduler queue backlogged onSchedulerBacklogged(manager) - clock.tick(schedulerBacklogTimeout * 1000 / 2) + clock.advance(schedulerBacklogTimeout * 1000 / 2) schedule(manager) assert(numExecutorsPending(manager) === 0) // timer not exceeded yet - clock.tick(schedulerBacklogTimeout * 1000) + clock.advance(schedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 1) // first timer exceeded - clock.tick(sustainedSchedulerBacklogTimeout * 1000 / 2) + clock.advance(sustainedSchedulerBacklogTimeout * 1000 / 2) schedule(manager) assert(numExecutorsPending(manager) === 1) // second timer not exceeded yet - clock.tick(sustainedSchedulerBacklogTimeout * 1000) + clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 1 + 2) // second timer exceeded - clock.tick(sustainedSchedulerBacklogTimeout * 1000) + clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 1 + 2 + 4) // third timer exceeded // Scheduler queue drained onSchedulerQueueEmpty(manager) - clock.tick(sustainedSchedulerBacklogTimeout * 1000) + clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 7) // timer is canceled - clock.tick(sustainedSchedulerBacklogTimeout * 1000) + clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 7) // Scheduler queue backlogged again onSchedulerBacklogged(manager) - clock.tick(schedulerBacklogTimeout * 1000) + clock.advance(schedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 7 + 1) // timer restarted - clock.tick(sustainedSchedulerBacklogTimeout * 1000) + clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 7 + 1 + 2) - clock.tick(sustainedSchedulerBacklogTimeout * 1000) + clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 7 + 1 + 2 + 4) - clock.tick(sustainedSchedulerBacklogTimeout * 1000) + clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 20) // limit reached } test("mock polling loop remove behavior") { sc = createSparkContext(1, 20) - val clock = new TestClock(2020L) + val clock = new ManualClock(2020L) val manager = sc.executorAllocationManager.get manager.setClock(clock) @@ -462,11 +487,11 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { onExecutorAdded(manager, "executor-3") assert(removeTimes(manager).size === 3) assert(executorsPendingToRemove(manager).isEmpty) - clock.tick(executorIdleTimeout * 1000 / 2) + clock.advance(executorIdleTimeout * 1000 / 2) schedule(manager) assert(removeTimes(manager).size === 3) // idle threshold not reached yet assert(executorsPendingToRemove(manager).isEmpty) - clock.tick(executorIdleTimeout * 1000) + clock.advance(executorIdleTimeout * 1000) schedule(manager) assert(removeTimes(manager).isEmpty) // idle threshold exceeded assert(executorsPendingToRemove(manager).size === 2) // limit reached (1 executor remaining) @@ -487,7 +512,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(!removeTimes(manager).contains("executor-5")) assert(!removeTimes(manager).contains("executor-6")) assert(executorsPendingToRemove(manager).size === 2) - clock.tick(executorIdleTimeout * 1000) + clock.advance(executorIdleTimeout * 1000) schedule(manager) assert(removeTimes(manager).isEmpty) // idle executors are removed assert(executorsPendingToRemove(manager).size === 4) @@ -505,7 +530,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).contains("executor-5")) assert(removeTimes(manager).contains("executor-6")) assert(executorsPendingToRemove(manager).size === 4) - clock.tick(executorIdleTimeout * 1000) + clock.advance(executorIdleTimeout * 1000) schedule(manager) assert(removeTimes(manager).isEmpty) assert(executorsPendingToRemove(manager).size === 6) // limit reached (1 executor remaining) @@ -579,30 +604,28 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).isEmpty) // New executors have registered - sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( - 0L, BlockManagerId("executor-1", "host1", 1), 100L)) + sc.listenerBus.postToAll(SparkListenerExecutorAdded( + 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) assert(executorIds(manager).size === 1) assert(executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 1) assert(removeTimes(manager).contains("executor-1")) - sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( - 0L, BlockManagerId("executor-2", "host2", 1), 100L)) + sc.listenerBus.postToAll(SparkListenerExecutorAdded( + 0L, "executor-2", new ExecutorInfo("host2", 1, Map.empty))) assert(executorIds(manager).size === 2) assert(executorIds(manager).contains("executor-2")) assert(removeTimes(manager).size === 2) assert(removeTimes(manager).contains("executor-2")) // Existing executors have disconnected - sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved( - 0L, BlockManagerId("executor-1", "host1", 1))) + sc.listenerBus.postToAll(SparkListenerExecutorRemoved(0L, "executor-1", "")) assert(executorIds(manager).size === 1) assert(!executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 1) assert(!removeTimes(manager).contains("executor-1")) // Unknown executor has disconnected - sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved( - 0L, BlockManagerId("executor-3", "host3", 1))) + sc.listenerBus.postToAll(SparkListenerExecutorRemoved(0L, "executor-3", "")) assert(executorIds(manager).size === 1) assert(removeTimes(manager).size === 1) } @@ -614,8 +637,8 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).isEmpty) sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) - sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( - 0L, BlockManagerId("executor-1", "host1", 1), 100L)) + sc.listenerBus.postToAll(SparkListenerExecutorAdded( + 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) assert(executorIds(manager).size === 1) assert(executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 0) @@ -626,16 +649,16 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { val manager = sc.executorAllocationManager.get assert(executorIds(manager).isEmpty) assert(removeTimes(manager).isEmpty) - sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( - 0L, BlockManagerId("executor-1", "host1", 1), 100L)) + sc.listenerBus.postToAll(SparkListenerExecutorAdded( + 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) assert(executorIds(manager).size === 1) assert(executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 0) - sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( - 0L, BlockManagerId("executor-2", "host1", 1), 100L)) + sc.listenerBus.postToAll(SparkListenerExecutorAdded( + 0L, "executor-2", new ExecutorInfo("host1", 1, Map.empty))) assert(executorIds(manager).size === 2) assert(executorIds(manager).contains("executor-2")) assert(removeTimes(manager).size === 1) @@ -682,6 +705,7 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { private val _numExecutorsToAdd = PrivateMethod[Int]('numExecutorsToAdd) private val _numExecutorsPending = PrivateMethod[Int]('numExecutorsPending) + private val _maxNumExecutorsNeeded = PrivateMethod[Int]('maxNumExecutorsNeeded) private val _executorsPendingToRemove = PrivateMethod[collection.Set[String]]('executorsPendingToRemove) private val _executorIds = PrivateMethod[collection.Set[String]]('executorIds) @@ -689,6 +713,7 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { private val _removeTimes = PrivateMethod[collection.Map[String, Long]]('removeTimes) private val _schedule = PrivateMethod[Unit]('schedule) private val _addExecutors = PrivateMethod[Int]('addExecutors) + private val _addOrCancelExecutorRequests = PrivateMethod[Int]('addOrCancelExecutorRequests) private val _removeExecutor = PrivateMethod[Boolean]('removeExecutor) private val _onExecutorAdded = PrivateMethod[Unit]('onExecutorAdded) private val _onExecutorRemoved = PrivateMethod[Unit]('onExecutorRemoved) @@ -727,7 +752,12 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { } private def addExecutors(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _addExecutors() + val maxNumExecutorsNeeded = manager invokePrivate _maxNumExecutorsNeeded() + manager invokePrivate _addExecutors(maxNumExecutorsNeeded) + } + + private def adjustRequestedExecutors(manager: ExecutorAllocationManager): Int = { + manager invokePrivate _addOrCancelExecutorRequests(0L) } private def removeExecutor(manager: ExecutorAllocationManager, id: String): Boolean = { diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 0f49ce4754fbb..5fdf6bc2777e3 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -18,13 +18,19 @@ package org.apache.spark import java.io._ +import java.net.URI import java.util.jar.{JarEntry, JarOutputStream} +import javax.net.ssl.SSLHandshakeException import com.google.common.io.ByteStreams +import org.apache.commons.io.{FileUtils, IOUtils} +import org.apache.commons.lang3.RandomUtils import org.scalatest.FunSuite import org.apache.spark.util.Utils +import SSLSampleConfigs._ + class FileServerSuite extends FunSuite with LocalSparkContext { @transient var tmpDir: File = _ @@ -168,4 +174,88 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } } + test ("HttpFileServer should work with SSL") { + val sparkConf = sparkSSLConfig() + val sm = new SecurityManager(sparkConf) + val server = new HttpFileServer(sparkConf, sm, 0) + try { + server.initialize() + + fileTransferTest(server, sm) + } finally { + server.stop() + } + } + + test ("HttpFileServer should work with SSL and good credentials") { + val sparkConf = sparkSSLConfig() + sparkConf.set("spark.authenticate", "true") + sparkConf.set("spark.authenticate.secret", "good") + + val sm = new SecurityManager(sparkConf) + val server = new HttpFileServer(sparkConf, sm, 0) + try { + server.initialize() + + fileTransferTest(server, sm) + } finally { + server.stop() + } + } + + test ("HttpFileServer should not work with valid SSL and bad credentials") { + val sparkConf = sparkSSLConfig() + sparkConf.set("spark.authenticate", "true") + sparkConf.set("spark.authenticate.secret", "bad") + + val sm = new SecurityManager(sparkConf) + val server = new HttpFileServer(sparkConf, sm, 0) + try { + server.initialize() + + intercept[IOException] { + fileTransferTest(server) + } + } finally { + server.stop() + } + } + + test ("HttpFileServer should not work with SSL when the server is untrusted") { + val sparkConf = sparkSSLConfigUntrusted() + val sm = new SecurityManager(sparkConf) + val server = new HttpFileServer(sparkConf, sm, 0) + try { + server.initialize() + + intercept[SSLHandshakeException] { + fileTransferTest(server) + } + } finally { + server.stop() + } + } + + def fileTransferTest(server: HttpFileServer, sm: SecurityManager = null): Unit = { + val randomContent = RandomUtils.nextBytes(100) + val file = File.createTempFile("FileServerSuite", "sslTests", tmpDir) + FileUtils.writeByteArrayToFile(file, randomContent) + server.addFile(file) + + val uri = new URI(server.serverUri + "/files/" + file.getName) + + val connection = if (sm != null && sm.isAuthenticationEnabled()) { + Utils.constructURIForAuthentication(uri, sm).toURL.openConnection() + } else { + uri.toURL.openConnection() + } + + if (sm != null) { + Utils.setupSecureURLConnection(connection, sm) + } + + val buf = IOUtils.toByteArray(connection.getInputStream) + assert(buf === randomContent) + } + } diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 5e24196101fbc..7acd27c735727 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -32,7 +32,6 @@ import org.apache.hadoop.mapreduce.lib.input.{FileSplit => NewFileSplit, TextInp import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} import org.scalatest.FunSuite -import org.apache.spark.SparkContext._ import org.apache.spark.rdd.{NewHadoopRDD, HadoopRDD} import org.apache.spark.util.Utils diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index d27880f4bc32f..ccfe0678cb1c3 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -120,7 +120,7 @@ class MapOutputTrackerSuite extends FunSuite { securityManager = new SecurityManager(conf)) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( - s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) val timeout = AkkaUtils.lookupTimeout(conf) slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) diff --git a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala new file mode 100644 index 0000000000000..444a33371bd71 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import java.io.File + +import com.google.common.io.Files +import org.apache.spark.util.Utils +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { + + test("test resolving property file as spark conf ") { + val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath + val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath + + val conf = new SparkConf + conf.set("spark.ssl.enabled", "true") + conf.set("spark.ssl.keyStore", keyStorePath) + conf.set("spark.ssl.keyStorePassword", "password") + conf.set("spark.ssl.keyPassword", "password") + conf.set("spark.ssl.trustStore", trustStorePath) + conf.set("spark.ssl.trustStorePassword", "password") + conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ssl.protocol", "SSLv3") + + val opts = SSLOptions.parse(conf, "spark.ssl") + + assert(opts.enabled === true) + assert(opts.trustStore.isDefined === true) + assert(opts.trustStore.get.getName === "truststore") + assert(opts.trustStore.get.getAbsolutePath === trustStorePath) + assert(opts.keyStore.isDefined === true) + assert(opts.keyStore.get.getName === "keystore") + assert(opts.keyStore.get.getAbsolutePath === keyStorePath) + assert(opts.trustStorePassword === Some("password")) + assert(opts.keyStorePassword === Some("password")) + assert(opts.keyPassword === Some("password")) + assert(opts.protocol === Some("SSLv3")) + assert(opts.enabledAlgorithms === Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) + } + + test("test resolving property with defaults specified ") { + val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath + val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath + + val conf = new SparkConf + conf.set("spark.ssl.enabled", "true") + conf.set("spark.ssl.keyStore", keyStorePath) + conf.set("spark.ssl.keyStorePassword", "password") + conf.set("spark.ssl.keyPassword", "password") + conf.set("spark.ssl.trustStore", trustStorePath) + conf.set("spark.ssl.trustStorePassword", "password") + conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ssl.protocol", "SSLv3") + + val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None) + val opts = SSLOptions.parse(conf, "spark.ui.ssl", defaults = Some(defaultOpts)) + + assert(opts.enabled === true) + assert(opts.trustStore.isDefined === true) + assert(opts.trustStore.get.getName === "truststore") + assert(opts.trustStore.get.getAbsolutePath === trustStorePath) + assert(opts.keyStore.isDefined === true) + assert(opts.keyStore.get.getName === "keystore") + assert(opts.keyStore.get.getAbsolutePath === keyStorePath) + assert(opts.trustStorePassword === Some("password")) + assert(opts.keyStorePassword === Some("password")) + assert(opts.keyPassword === Some("password")) + assert(opts.protocol === Some("SSLv3")) + assert(opts.enabledAlgorithms === Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) + } + + test("test whether defaults can be overridden ") { + val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath + val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath + + val conf = new SparkConf + conf.set("spark.ssl.enabled", "true") + conf.set("spark.ui.ssl.enabled", "false") + conf.set("spark.ssl.keyStore", keyStorePath) + conf.set("spark.ssl.keyStorePassword", "password") + conf.set("spark.ui.ssl.keyStorePassword", "12345") + conf.set("spark.ssl.keyPassword", "password") + conf.set("spark.ssl.trustStore", trustStorePath) + conf.set("spark.ssl.trustStorePassword", "password") + conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ui.ssl.enabledAlgorithms", "ABC, DEF") + conf.set("spark.ssl.protocol", "SSLv3") + + val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None) + val opts = SSLOptions.parse(conf, "spark.ui.ssl", defaults = Some(defaultOpts)) + + assert(opts.enabled === false) + assert(opts.trustStore.isDefined === true) + assert(opts.trustStore.get.getName === "truststore") + assert(opts.trustStore.get.getAbsolutePath === trustStorePath) + assert(opts.keyStore.isDefined === true) + assert(opts.keyStore.get.getName === "keystore") + assert(opts.keyStore.get.getAbsolutePath === keyStorePath) + assert(opts.trustStorePassword === Some("password")) + assert(opts.keyStorePassword === Some("12345")) + assert(opts.keyPassword === Some("password")) + assert(opts.protocol === Some("SSLv3")) + assert(opts.enabledAlgorithms === Set("ABC", "DEF")) + } + +} diff --git a/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala new file mode 100644 index 0000000000000..ace8123a8961f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import java.io.File + +object SSLSampleConfigs { + val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath + val untrustedKeyStorePath = new File(this.getClass.getResource("/untrusted-keystore").toURI).getAbsolutePath + val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath + + def sparkSSLConfig() = { + val conf = new SparkConf(loadDefaults = false) + conf.set("spark.ssl.enabled", "true") + conf.set("spark.ssl.keyStore", keyStorePath) + conf.set("spark.ssl.keyStorePassword", "password") + conf.set("spark.ssl.keyPassword", "password") + conf.set("spark.ssl.trustStore", trustStorePath) + conf.set("spark.ssl.trustStorePassword", "password") + conf.set("spark.ssl.enabledAlgorithms", + "TLS_RSA_WITH_AES_128_CBC_SHA, SSL_RSA_WITH_DES_CBC_SHA") + conf.set("spark.ssl.protocol", "TLSv1") + conf + } + + def sparkSSLConfigUntrusted() = { + val conf = new SparkConf(loadDefaults = false) + conf.set("spark.ssl.enabled", "true") + conf.set("spark.ssl.keyStore", untrustedKeyStorePath) + conf.set("spark.ssl.keyStorePassword", "password") + conf.set("spark.ssl.keyPassword", "password") + conf.set("spark.ssl.trustStore", trustStorePath) + conf.set("spark.ssl.trustStorePassword", "password") + conf.set("spark.ssl.enabledAlgorithms", + "TLS_RSA_WITH_AES_128_CBC_SHA, SSL_RSA_WITH_DES_CBC_SHA") + conf.set("spark.ssl.protocol", "TLSv1") + conf + } + +} diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index fcca0867b8072..43fbd3ff3f756 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark -import scala.collection.mutable.ArrayBuffer +import java.io.File import org.scalatest.FunSuite @@ -125,6 +125,54 @@ class SecurityManagerSuite extends FunSuite { } + test("ssl on setup") { + val conf = SSLSampleConfigs.sparkSSLConfig() + + val securityManager = new SecurityManager(conf) + + assert(securityManager.fileServerSSLOptions.enabled === true) + assert(securityManager.akkaSSLOptions.enabled === true) + + assert(securityManager.sslSocketFactory.isDefined === true) + assert(securityManager.hostnameVerifier.isDefined === true) + + assert(securityManager.fileServerSSLOptions.trustStore.isDefined === true) + assert(securityManager.fileServerSSLOptions.trustStore.get.getName === "truststore") + assert(securityManager.fileServerSSLOptions.keyStore.isDefined === true) + assert(securityManager.fileServerSSLOptions.keyStore.get.getName === "keystore") + assert(securityManager.fileServerSSLOptions.trustStorePassword === Some("password")) + assert(securityManager.fileServerSSLOptions.keyStorePassword === Some("password")) + assert(securityManager.fileServerSSLOptions.keyPassword === Some("password")) + assert(securityManager.fileServerSSLOptions.protocol === Some("TLSv1")) + assert(securityManager.fileServerSSLOptions.enabledAlgorithms === + Set("TLS_RSA_WITH_AES_128_CBC_SHA", "SSL_RSA_WITH_DES_CBC_SHA")) + + assert(securityManager.akkaSSLOptions.trustStore.isDefined === true) + assert(securityManager.akkaSSLOptions.trustStore.get.getName === "truststore") + assert(securityManager.akkaSSLOptions.keyStore.isDefined === true) + assert(securityManager.akkaSSLOptions.keyStore.get.getName === "keystore") + assert(securityManager.akkaSSLOptions.trustStorePassword === Some("password")) + assert(securityManager.akkaSSLOptions.keyStorePassword === Some("password")) + assert(securityManager.akkaSSLOptions.keyPassword === Some("password")) + assert(securityManager.akkaSSLOptions.protocol === Some("TLSv1")) + assert(securityManager.akkaSSLOptions.enabledAlgorithms === + Set("TLS_RSA_WITH_AES_128_CBC_SHA", "SSL_RSA_WITH_DES_CBC_SHA")) + } + + test("ssl off setup") { + val file = File.createTempFile("SSLOptionsSuite", "conf") + file.deleteOnExit() + + System.setProperty("spark.ssl.configFile", file.getAbsolutePath) + val conf = new SparkConf() + + val securityManager = new SecurityManager(conf) + + assert(securityManager.fileServerSSLOptions.enabled === false) + assert(securityManager.akkaSSLOptions.enabled === false) + assert(securityManager.sslSocketFactory.isDefined === false) + assert(securityManager.hostnameVerifier.isDefined === false) + } } diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 790976a5ac308..e08210ae60d17 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -17,6 +17,10 @@ package org.apache.spark +import java.util.concurrent.{TimeUnit, Executors} + +import scala.util.{Try, Random} + import org.scalatest.FunSuite import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer} import org.apache.spark.util.ResetSystemProperties @@ -123,6 +127,27 @@ class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemPro assert(conf.get("spark.test.a.b.c") === "a.b.c") } + test("Thread safeness - SPARK-5425") { + import scala.collection.JavaConversions._ + val executor = Executors.newSingleThreadScheduledExecutor() + val sf = executor.scheduleAtFixedRate(new Runnable { + override def run(): Unit = + System.setProperty("spark.5425." + Random.nextInt(), Random.nextInt().toString) + }, 0, 1, TimeUnit.MILLISECONDS) + + try { + val t0 = System.currentTimeMillis() + while ((System.currentTimeMillis() - t0) < 1000) { + val conf = Try(new SparkConf(loadDefaults = true)) + assert(conf.isSuccess === true) + } + } finally { + executor.shutdownNow() + for (key <- System.getProperties.stringPropertyNames() if key.startsWith("spark.5425.")) + System.getProperties.remove(key) + } + } + test("register kryo classes through registerKryoClasses") { val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 8b3c6871a7b39..50f347f1954de 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -17,10 +17,17 @@ package org.apache.spark +import java.io.File + +import com.google.common.base.Charsets._ +import com.google.common.io.Files + import org.scalatest.FunSuite import org.apache.hadoop.io.BytesWritable +import org.apache.spark.util.Utils + class SparkContextSuite extends FunSuite with LocalSparkContext { test("Only one SparkContext may be active at a time") { @@ -72,4 +79,74 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { val byteArray2 = converter.convert(bytesWritable) assert(byteArray2.length === 0) } + + test("addFile works") { + val file = File.createTempFile("someprefix", "somesuffix") + val absolutePath = file.getAbsolutePath + try { + Files.write("somewords", file, UTF_8) + val length = file.length() + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addFile(file.getAbsolutePath) + sc.parallelize(Array(1), 1).map(x => { + val gotten = new File(SparkFiles.get(file.getName)) + if (!gotten.exists()) { + throw new SparkException("file doesn't exist") + } + if (length != gotten.length()) { + throw new SparkException( + s"file has different length $length than added file ${gotten.length()}") + } + if (absolutePath == gotten.getAbsolutePath) { + throw new SparkException("file should have been copied") + } + x + }).count() + } finally { + sc.stop() + } + } + + test("addFile recursive works") { + val pluto = Utils.createTempDir() + val neptune = Utils.createTempDir(pluto.getAbsolutePath) + val saturn = Utils.createTempDir(neptune.getAbsolutePath) + val alien1 = File.createTempFile("alien", "1", neptune) + val alien2 = File.createTempFile("alien", "2", saturn) + + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addFile(neptune.getAbsolutePath, true) + sc.parallelize(Array(1), 1).map(x => { + val sep = File.separator + if (!new File(SparkFiles.get(neptune.getName + sep + alien1.getName)).exists()) { + throw new SparkException("can't access file under root added directory") + } + if (!new File(SparkFiles.get(neptune.getName + sep + saturn.getName + sep + alien2.getName)) + .exists()) { + throw new SparkException("can't access file in nested directory") + } + if (new File(SparkFiles.get(pluto.getName + sep + neptune.getName + sep + alien1.getName)) + .exists()) { + throw new SparkException("file exists that shouldn't") + } + x + }).count() + } finally { + sc.stop() + } + } + + test("addFile recursive can't add directories by default") { + val dir = Utils.createTempDir() + + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + intercept[SparkException] { + sc.addFile(dir.getAbsolutePath) + } + } finally { + sc.stop() + } + } } diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala index 7b866f08a0e9f..c63d834f9048b 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala @@ -23,11 +23,22 @@ import org.scalatest.FunSuite class PythonRDDSuite extends FunSuite { - test("Writing large strings to the worker") { - val input: List[String] = List("a"*100000) - val buffer = new DataOutputStream(new ByteArrayOutputStream) - PythonRDD.writeIteratorToStream(input.iterator, buffer) - } + test("Writing large strings to the worker") { + val input: List[String] = List("a"*100000) + val buffer = new DataOutputStream(new ByteArrayOutputStream) + PythonRDD.writeIteratorToStream(input.iterator, buffer) + } + test("Handle nulls gracefully") { + val buffer = new DataOutputStream(new ByteArrayOutputStream) + // Should not have NPE when write an Iterator with null in it + // The correctness will be tested in Python + PythonRDD.writeIteratorToStream(Iterator("a", null), buffer) + PythonRDD.writeIteratorToStream(Iterator(null, "a"), buffer) + PythonRDD.writeIteratorToStream(Iterator("a".getBytes, null), buffer) + PythonRDD.writeIteratorToStream(Iterator(null, "a".getBytes), buffer) + PythonRDD.writeIteratorToStream(Iterator((null, null), ("a", null), (null, "b")), buffer) + PythonRDD.writeIteratorToStream( + Iterator((null, null), ("a".getBytes, null), (null, "b".getBytes)), buffer) + } } - diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala index d2dae34be7bfb..518073dcbb64e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala @@ -23,6 +23,7 @@ import org.scalatest.Matchers class ClientSuite extends FunSuite with Matchers { test("correctly validates driver jar URL's") { ClientArguments.isValidJarUrl("http://someHost:8080/foo.jar") should be (true) + ClientArguments.isValidJarUrl("https://someHost:8080/foo.jar") should be (true) // file scheme with authority and path is valid. ClientArguments.isValidJarUrl("file://somehost/path/to/a/jarFile.jar") should be (true) diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index aa65f7e8915e6..68b5776fc6515 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -68,7 +68,8 @@ class JsonProtocolSuite extends FunSuite { val completedApps = Array[ApplicationInfo]() val activeDrivers = Array(createDriverInfo()) val completedDrivers = Array(createDriverInfo()) - val stateResponse = new MasterStateResponse("host", 8080, workers, activeApps, completedApps, + val stateResponse = new MasterStateResponse( + "host", 8080, None, workers, activeApps, completedApps, activeDrivers, completedDrivers, RecoveryState.ALIVE) val output = JsonProtocol.writeMasterState(stateResponse) assertValidJson(output) @@ -117,8 +118,8 @@ class JsonProtocolSuite extends FunSuite { } def createExecutorRunner(): ExecutorRunner = { - new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", - new File("sparkHome"), new File("workDir"), "akka://worker", + new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", 123, + "publicAddress", new File("sparkHome"), new File("workDir"), "akka://worker", new SparkConf, Seq("localDir"), ExecutorState.RUNNING) } diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala new file mode 100644 index 0000000000000..54dd7c9c45c61 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala @@ -0,0 +1,93 @@ +/* + * 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 + +import java.net.URL + +import scala.collection.mutable +import scala.io.Source + +import org.scalatest.FunSuite + +import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.scheduler.{SparkListenerExecutorAdded, SparkListener} +import org.apache.spark.{SparkConf, SparkContext, LocalSparkContext} + +class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext { + + /** Length of time to wait while draining listener events. */ + private val WAIT_TIMEOUT_MILLIS = 10000 + + test("verify that correct log urls get propagated from workers") { + sc = new SparkContext("local-cluster[2,1,512]", "test") + + val listener = new SaveExecutorInfo + sc.addSparkListener(listener) + + // Trigger a job so that executors get added + sc.parallelize(1 to 100, 4).map(_.toString).count() + + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + listener.addedExecutorInfos.values.foreach { info => + assert(info.logUrlMap.nonEmpty) + // Browse to each URL to check that it's valid + info.logUrlMap.foreach { case (logType, logUrl) => + val html = Source.fromURL(logUrl).mkString + assert(html.contains(s"$logType log page")) + } + } + } + + test("verify that log urls reflect SPARK_PUBLIC_DNS (SPARK-6175)") { + val SPARK_PUBLIC_DNS = "public_dns" + class MySparkConf extends SparkConf(false) { + override def getenv(name: String) = { + if (name == "SPARK_PUBLIC_DNS") SPARK_PUBLIC_DNS + else super.getenv(name) + } + + override def clone: SparkConf = { + new MySparkConf().setAll(getAll) + } + } + val conf = new MySparkConf() + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) + + val listener = new SaveExecutorInfo + sc.addSparkListener(listener) + + // Trigger a job so that executors get added + sc.parallelize(1 to 100, 4).map(_.toString).count() + + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + listener.addedExecutorInfos.values.foreach { info => + assert(info.logUrlMap.nonEmpty) + info.logUrlMap.values.foreach { logUrl => + assert(new URL(logUrl).getHost === SPARK_PUBLIC_DNS) + } + } + } + + private class SaveExecutorInfo extends SparkListener { + val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() + + override def onExecutorAdded(executor: SparkListenerExecutorAdded) { + addedExecutorInfos(executor.executorId) = executor.executorInfo + } + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 82628ad3abd99..46d745c4ecbfa 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -21,6 +21,8 @@ import java.io._ import scala.collection.mutable.ArrayBuffer +import com.google.common.base.Charsets.UTF_8 +import com.google.common.io.ByteStreams import org.scalatest.FunSuite import org.scalatest.Matchers import org.scalatest.concurrent.Timeouts @@ -141,7 +143,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) - val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) val childArgsStr = childArgs.mkString(" ") childArgsStr should include ("--class org.SomeClass") childArgsStr should include ("--executor-memory 5g") @@ -180,7 +182,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) - val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") mainClass should be ("org.SomeClass") classpath should have length (4) @@ -201,6 +203,18 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties } test("handles standalone cluster mode") { + testStandaloneCluster(useRest = true) + } + + test("handles legacy standalone cluster mode") { + testStandaloneCluster(useRest = false) + } + + /** + * Test whether the launch environment is correctly set up in standalone cluster mode. + * @param useRest whether to use the REST submission gateway introduced in Spark 1.3 + */ + private def testStandaloneCluster(useRest: Boolean): Unit = { val clArgs = Seq( "--deploy-mode", "cluster", "--master", "spark://h:p", @@ -212,17 +226,26 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) - val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + appArgs.useRest = useRest + val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) val childArgsStr = childArgs.mkString(" ") - childArgsStr should startWith ("--memory 4g --cores 5 --supervise") - childArgsStr should include regex ("launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2") - mainClass should be ("org.apache.spark.deploy.Client") - classpath should have size (0) - sysProps should have size (5) + if (useRest) { + childArgsStr should endWith ("thejar.jar org.SomeClass arg1 arg2") + mainClass should be ("org.apache.spark.deploy.rest.StandaloneRestClient") + } else { + childArgsStr should startWith ("--supervise --memory 4g --cores 5") + childArgsStr should include regex "launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2" + mainClass should be ("org.apache.spark.deploy.Client") + } + classpath should have size 0 + sysProps should have size 8 sysProps.keys should contain ("SPARK_SUBMIT") sysProps.keys should contain ("spark.master") sysProps.keys should contain ("spark.app.name") sysProps.keys should contain ("spark.jars") + sysProps.keys should contain ("spark.driver.memory") + sysProps.keys should contain ("spark.driver.cores") + sysProps.keys should contain ("spark.driver.supervise") sysProps.keys should contain ("spark.shuffle.spill") sysProps("spark.shuffle.spill") should be ("false") } @@ -239,7 +262,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) - val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") mainClass should be ("org.SomeClass") classpath should have length (1) @@ -261,7 +284,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) - val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") mainClass should be ("org.SomeClass") classpath should have length (1) @@ -281,7 +304,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) - val (_, _, sysProps, mainClass) = createLaunchEnv(appArgs) + val (_, _, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) sysProps("spark.executor.memory") should be ("5g") sysProps("spark.master") should be ("yarn-cluster") mainClass should be ("org.apache.spark.deploy.yarn.Client") @@ -307,7 +330,21 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "--name", "testApp", "--master", "local-cluster[2,1,512]", "--jars", jarsString, - unusedJar.toString) + unusedJar.toString, "SparkSubmitClassA", "SparkSubmitClassB") + runSparkSubmit(args) + } + + test("includes jars passed in through --packages") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val packagesString = "com.databricks:spark-csv_2.10:0.1,com.databricks:spark-avro_2.10:0.1" + val args = Seq( + "--class", JarCreationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local-cluster[2,1,512]", + "--packages", packagesString, + "--conf", "spark.ui.enabled=false", + unusedJar.toString, + "com.databricks.spark.csv.DefaultSource", "com.databricks.spark.avro.DefaultSource") runSparkSubmit(args) } @@ -325,7 +362,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "--files", files, "thejar.jar") val appArgs = new SparkSubmitArguments(clArgs) - val sysProps = SparkSubmit.createLaunchEnv(appArgs)._3 + val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs)._3 appArgs.jars should be (Utils.resolveURIs(jars)) appArgs.files should be (Utils.resolveURIs(files)) sysProps("spark.jars") should be (Utils.resolveURIs(jars + ",thejar.jar")) @@ -340,7 +377,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar" ) val appArgs2 = new SparkSubmitArguments(clArgs2) - val sysProps2 = SparkSubmit.createLaunchEnv(appArgs2)._3 + val sysProps2 = SparkSubmit.prepareSubmitEnvironment(appArgs2)._3 appArgs2.files should be (Utils.resolveURIs(files)) appArgs2.archives should be (Utils.resolveURIs(archives)) sysProps2("spark.yarn.dist.files") should be (Utils.resolveURIs(files)) @@ -353,7 +390,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "mister.py" ) val appArgs3 = new SparkSubmitArguments(clArgs3) - val sysProps3 = SparkSubmit.createLaunchEnv(appArgs3)._3 + val sysProps3 = SparkSubmit.prepareSubmitEnvironment(appArgs3)._3 appArgs3.pyFiles should be (Utils.resolveURIs(pyFiles)) sysProps3("spark.submit.pyFiles") should be ( PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) @@ -378,7 +415,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar" ) val appArgs = new SparkSubmitArguments(clArgs) - val sysProps = SparkSubmit.createLaunchEnv(appArgs)._3 + val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs)._3 sysProps("spark.jars") should be(Utils.resolveURIs(jars + ",thejar.jar")) sysProps("spark.files") should be(Utils.resolveURIs(files)) @@ -395,7 +432,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar" ) val appArgs2 = new SparkSubmitArguments(clArgs2) - val sysProps2 = SparkSubmit.createLaunchEnv(appArgs2)._3 + val sysProps2 = SparkSubmit.prepareSubmitEnvironment(appArgs2)._3 sysProps2("spark.yarn.dist.files") should be(Utils.resolveURIs(files)) sysProps2("spark.yarn.dist.archives") should be(Utils.resolveURIs(archives)) @@ -410,11 +447,24 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "mister.py" ) val appArgs3 = new SparkSubmitArguments(clArgs3) - val sysProps3 = SparkSubmit.createLaunchEnv(appArgs3)._3 + val sysProps3 = SparkSubmit.prepareSubmitEnvironment(appArgs3)._3 sysProps3("spark.submit.pyFiles") should be( PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) } + test("user classpath first in driver") { + val systemJar = TestUtils.createJarWithFiles(Map("test.resource" -> "SYSTEM")) + val userJar = TestUtils.createJarWithFiles(Map("test.resource" -> "USER")) + val args = Seq( + "--class", UserClasspathFirstTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local", + "--conf", "spark.driver.extraClassPath=" + systemJar, + "--conf", "spark.driver.userClassPathFirst=true", + userJar.toString) + runSparkSubmit(args) + } + test("SPARK_CONF_DIR overrides spark-defaults.conf") { forConfDir(Map("spark.executor.memory" -> "2.3g")) { path => val unusedJar = TestUtils.createJarWithClasses(Seq.empty) @@ -426,7 +476,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties val appArgs = new SparkSubmitArguments(args, Map("SPARK_CONF_DIR" -> path)) assert(appArgs.propertiesFile != null) assert(appArgs.propertiesFile.startsWith(path)) - appArgs.executorMemory should be ("2.3g") + appArgs.executorMemory should be ("2.3g") } } @@ -467,8 +517,8 @@ object JarCreationTest extends Logging { val result = sc.makeRDD(1 to 100, 10).mapPartitions { x => var exception: String = null try { - Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) - Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) + Class.forName(args(0), true, Thread.currentThread().getContextClassLoader) + Class.forName(args(1), true, Thread.currentThread().getContextClassLoader) } catch { case t: Throwable => exception = t + "\n" + t.getStackTraceString @@ -506,3 +556,15 @@ object SimpleApplicationTest { } } } + +object UserClasspathFirstTest { + def main(args: Array[String]) { + val ccl = Thread.currentThread().getContextClassLoader() + val resource = ccl.getResourceAsStream("test.resource") + val bytes = ByteStreams.toByteArray(resource) + val contents = new String(bytes, 0, bytes.length, UTF_8) + if (contents != "USER") { + throw new SparkException("Should have read user resource, but instead read: " + contents) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala new file mode 100644 index 0000000000000..8bcca926097a1 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -0,0 +1,136 @@ +/* + * 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 + +import java.io.{PrintStream, OutputStream, File} + +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.ivy.core.module.descriptor.MDArtifact +import org.apache.ivy.plugins.resolver.IBiblioResolver + +class SparkSubmitUtilsSuite extends FunSuite with BeforeAndAfterAll { + + private val noOpOutputStream = new OutputStream { + def write(b: Int) = {} + } + + /** Simple PrintStream that reads data into a buffer */ + private class BufferPrintStream extends PrintStream(noOpOutputStream) { + var lineBuffer = ArrayBuffer[String]() + override def println(line: String) { + lineBuffer += line + } + } + + override def beforeAll() { + super.beforeAll() + // We don't want to write logs during testing + SparkSubmitUtils.printStream = new BufferPrintStream + } + + test("incorrect maven coordinate throws error") { + val coordinates = Seq("a:b: ", " :a:b", "a: :b", "a:b:", ":a:b", "a::b", "::", "a:b", "a") + for (coordinate <- coordinates) { + intercept[IllegalArgumentException] { + SparkSubmitUtils.extractMavenCoordinates(coordinate) + } + } + } + + test("create repo resolvers") { + val resolver1 = SparkSubmitUtils.createRepoResolvers(None) + // should have central and spark-packages by default + assert(resolver1.getResolvers.size() === 2) + assert(resolver1.getResolvers.get(0).asInstanceOf[IBiblioResolver].getName === "central") + assert(resolver1.getResolvers.get(1).asInstanceOf[IBiblioResolver].getName === "spark-packages") + + val repos = "a/1,b/2,c/3" + val resolver2 = SparkSubmitUtils.createRepoResolvers(Option(repos)) + assert(resolver2.getResolvers.size() === 5) + val expected = repos.split(",").map(r => s"$r/") + resolver2.getResolvers.toArray.zipWithIndex.foreach { case (resolver: IBiblioResolver, i) => + if (i == 0) { + assert(resolver.getName === "central") + } else if (i == 1) { + assert(resolver.getName === "spark-packages") + } else { + assert(resolver.getName === s"repo-${i - 1}") + assert(resolver.getRoot === expected(i - 2)) + } + } + } + + test("add dependencies works correctly") { + val md = SparkSubmitUtils.getModuleDescriptor + val artifacts = SparkSubmitUtils.extractMavenCoordinates("com.databricks:spark-csv_2.10:0.1," + + "com.databricks:spark-avro_2.10:0.1") + + SparkSubmitUtils.addDependenciesToIvy(md, artifacts, "default") + assert(md.getDependencies.length === 2) + } + + test("ivy path works correctly") { + val ivyPath = "dummy/ivy" + val md = SparkSubmitUtils.getModuleDescriptor + val artifacts = for (i <- 0 until 3) yield new MDArtifact(md, s"jar-$i", "jar", "jar") + var jPaths = SparkSubmitUtils.resolveDependencyPaths(artifacts.toArray, new File(ivyPath)) + for (i <- 0 until 3) { + val index = jPaths.indexOf(ivyPath) + assert(index >= 0) + jPaths = jPaths.substring(index + ivyPath.length) + } + // end to end + val jarPath = SparkSubmitUtils.resolveMavenCoordinates( + "com.databricks:spark-csv_2.10:0.1", None, Option(ivyPath), true) + assert(jarPath.indexOf(ivyPath) >= 0, "should use non-default ivy path") + } + + test("search for artifact at other repositories") { + val path = SparkSubmitUtils.resolveMavenCoordinates("com.agimatec:agimatec-validation:0.9.3", + Option("https://oss.sonatype.org/content/repositories/agimatec/"), None, true) + assert(path.indexOf("agimatec-validation") >= 0, "should find package. If it doesn't, check" + + "if package still exists. If it has been removed, replace the example in this test.") + } + + test("dependency not found throws RuntimeException") { + intercept[RuntimeException] { + SparkSubmitUtils.resolveMavenCoordinates("a:b:c", None, None, true) + } + } + + test("neglects Spark and Spark's dependencies") { + val components = Seq("bagel_", "catalyst_", "core_", "graphx_", "hive_", "mllib_", "repl_", + "sql_", "streaming_", "yarn_", "network-common_", "network-shuffle_", "network-yarn_") + + val coordinates = + components.map(comp => s"org.apache.spark:spark-${comp}2.10:1.2.0").mkString(",") + + ",org.apache.spark:spark-core_fake:1.2.0" + + val path = SparkSubmitUtils.resolveMavenCoordinates(coordinates, None, None, true) + assert(path === "", "should return empty path") + // Should not exclude the following dependency. Will throw an error, because it doesn't exist, + // but the fact that it is checking means that it wasn't excluded. + intercept[RuntimeException] { + SparkSubmitUtils.resolveMavenCoordinates(coordinates + + ",org.apache.spark:spark-streaming-kafka-assembly_2.10:1.2.0", None, None, true) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 3fbc1a21d10ed..e908ba604ebed 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -17,18 +17,17 @@ package org.apache.spark.deploy.history -import java.io.{File, FileOutputStream, OutputStreamWriter} +import java.io.{BufferedOutputStream, File, FileOutputStream, OutputStreamWriter} +import java.net.URI import scala.io.Source -import com.google.common.io.Files import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.Matchers import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io._ import org.apache.spark.scheduler._ import org.apache.spark.util.{JsonProtocol, Utils} @@ -37,54 +36,67 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers private var testDir: File = null - private var provider: FsHistoryProvider = null - before { testDir = Utils.createTempDir() - provider = new FsHistoryProvider(new SparkConf() - .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) - .set("spark.history.fs.updateInterval", "0")) } after { Utils.deleteRecursively(testDir) } + /** Create a fake log file using the new log format used in Spark 1.3+ */ + private def newLogFile( + appId: String, + inProgress: Boolean, + codec: Option[String] = None): File = { + val ip = if (inProgress) EventLoggingListener.IN_PROGRESS else "" + val logUri = EventLoggingListener.getLogPath(testDir.getAbsolutePath, appId) + val logPath = new URI(logUri).getPath + ip + new File(logPath) + } + test("Parse new and old application logs") { - val conf = new SparkConf() - .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) - .set("spark.history.fs.updateInterval", "0") - val provider = new FsHistoryProvider(conf) + val provider = new FsHistoryProvider(createTestConf()) // Write a new-style application log. - val logFile1 = new File(testDir, "new1") - writeFile(logFile1, true, None, - SparkListenerApplicationStart("app1-1", None, 1L, "test"), - SparkListenerApplicationEnd(2L) + val newAppComplete = newLogFile("new1", inProgress = false) + writeFile(newAppComplete, true, None, + SparkListenerApplicationStart("new-app-complete", None, 1L, "test"), + SparkListenerApplicationEnd(5L) ) + // Write a new-style application log. + val newAppCompressedComplete = newLogFile("new1compressed", inProgress = false, Some("lzf")) + writeFile(newAppCompressedComplete, true, None, + SparkListenerApplicationStart("new-app-compressed-complete", None, 1L, "test"), + SparkListenerApplicationEnd(4L)) + // Write an unfinished app, new-style. - val logFile2 = new File(testDir, "new2" + EventLoggingListener.IN_PROGRESS) - writeFile(logFile2, true, None, - SparkListenerApplicationStart("app2-2", None, 1L, "test") + val newAppIncomplete = newLogFile("new2", inProgress = true) + writeFile(newAppIncomplete, true, None, + SparkListenerApplicationStart("new-app-incomplete", None, 1L, "test") ) // Write an old-style application log. - val oldLog = new File(testDir, "old1") - oldLog.mkdir() - createEmptyFile(new File(oldLog, provider.SPARK_VERSION_PREFIX + "1.0")) - writeFile(new File(oldLog, provider.LOG_PREFIX + "1"), false, None, - SparkListenerApplicationStart("app3", None, 2L, "test"), + val oldAppComplete = new File(testDir, "old1") + oldAppComplete.mkdir() + createEmptyFile(new File(oldAppComplete, provider.SPARK_VERSION_PREFIX + "1.0")) + writeFile(new File(oldAppComplete, provider.LOG_PREFIX + "1"), false, None, + SparkListenerApplicationStart("old-app-complete", None, 2L, "test"), SparkListenerApplicationEnd(3L) ) - createEmptyFile(new File(oldLog, provider.APPLICATION_COMPLETE)) + createEmptyFile(new File(oldAppComplete, provider.APPLICATION_COMPLETE)) + + // Check for logs so that we force the older unfinished app to be loaded, to make + // sure unfinished apps are also sorted correctly. + provider.checkForLogs() // Write an unfinished app, old-style. - val oldLog2 = new File(testDir, "old2") - oldLog2.mkdir() - createEmptyFile(new File(oldLog2, provider.SPARK_VERSION_PREFIX + "1.0")) - writeFile(new File(oldLog2, provider.LOG_PREFIX + "1"), false, None, - SparkListenerApplicationStart("app4", None, 2L, "test") + val oldAppIncomplete = new File(testDir, "old2") + oldAppIncomplete.mkdir() + createEmptyFile(new File(oldAppIncomplete, provider.SPARK_VERSION_PREFIX + "1.0")) + writeFile(new File(oldAppIncomplete, provider.LOG_PREFIX + "1"), false, None, + SparkListenerApplicationStart("old-app-incomplete", None, 2L, "test") ) // Force a reload of data from the log directory, and check that both logs are loaded. @@ -93,17 +105,19 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val list = provider.getListing().toSeq list should not be (null) - list.size should be (4) - list.count(e => e.completed) should be (2) - - list(0) should be (ApplicationHistoryInfo(oldLog.getName(), "app3", 2L, 3L, - oldLog.lastModified(), "test", true)) - list(1) should be (ApplicationHistoryInfo(logFile1.getName(), "app1-1", 1L, 2L, - logFile1.lastModified(), "test", true)) - list(2) should be (ApplicationHistoryInfo(oldLog2.getName(), "app4", 2L, -1L, - oldLog2.lastModified(), "test", false)) - list(3) should be (ApplicationHistoryInfo(logFile2.getName(), "app2-2", 1L, -1L, - logFile2.lastModified(), "test", false)) + list.size should be (5) + list.count(_.completed) should be (3) + + list(0) should be (ApplicationHistoryInfo(newAppComplete.getName(), "new-app-complete", 1L, 5L, + newAppComplete.lastModified(), "test", true)) + list(1) should be (ApplicationHistoryInfo(newAppCompressedComplete.getName(), + "new-app-compressed-complete", 1L, 4L, newAppCompressedComplete.lastModified(), "test", true)) + list(2) should be (ApplicationHistoryInfo(oldAppComplete.getName(), "old-app-complete", 2L, 3L, + oldAppComplete.lastModified(), "test", true)) + list(3) should be (ApplicationHistoryInfo(oldAppIncomplete.getName(), "old-app-incomplete", 2L, + -1L, oldAppIncomplete.lastModified(), "test", false)) + list(4) should be (ApplicationHistoryInfo(newAppIncomplete.getName(), "new-app-incomplete", 1L, + -1L, newAppIncomplete.lastModified(), "test", false)) // Make sure the UI can be rendered. list.foreach { case info => @@ -113,6 +127,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers } test("Parse legacy logs with compression codec set") { + val provider = new FsHistoryProvider(createTestConf()) val testCodecs = List((classOf[LZFCompressionCodec].getName(), true), (classOf[SnappyCompressionCodec].getName(), true), ("invalid.codec", false)) @@ -130,7 +145,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val logPath = new Path(logDir.getAbsolutePath()) try { - val (logInput, sparkVersion) = provider.openLegacyEventLog(logPath) + val logInput = provider.openLegacyEventLog(logPath) try { Source.fromInputStream(logInput).getLines().toSeq.size should be (2) } finally { @@ -144,22 +159,19 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers } test("SPARK-3697: ignore directories that cannot be read.") { - val logFile1 = new File(testDir, "new1") + val logFile1 = newLogFile("new1", inProgress = false) writeFile(logFile1, true, None, SparkListenerApplicationStart("app1-1", None, 1L, "test"), SparkListenerApplicationEnd(2L) ) - val logFile2 = new File(testDir, "new2") + val logFile2 = newLogFile("new2", inProgress = false) writeFile(logFile2, true, None, SparkListenerApplicationStart("app1-2", None, 1L, "test"), SparkListenerApplicationEnd(2L) ) logFile2.setReadable(false, false) - val conf = new SparkConf() - .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) - .set("spark.history.fs.updateInterval", "0") - val provider = new FsHistoryProvider(conf) + val provider = new FsHistoryProvider(createTestConf()) provider.checkForLogs() val list = provider.getListing().toSeq @@ -168,12 +180,9 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers } test("history file is renamed from inprogress to completed") { - val conf = new SparkConf() - .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) - .set("spark.testing", "true") - val provider = new FsHistoryProvider(conf) + val provider = new FsHistoryProvider(createTestConf()) - val logFile1 = new File(testDir, "app1" + EventLoggingListener.IN_PROGRESS) + val logFile1 = newLogFile("app1", inProgress = true) writeFile(logFile1, true, None, SparkListenerApplicationStart("app1", Some("app1"), 1L, "test"), SparkListenerApplicationEnd(2L) @@ -183,23 +192,38 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers appListBeforeRename.size should be (1) appListBeforeRename.head.logPath should endWith(EventLoggingListener.IN_PROGRESS) - logFile1.renameTo(new File(testDir, "app1")) + logFile1.renameTo(newLogFile("app1", inProgress = false)) provider.checkForLogs() val appListAfterRename = provider.getListing() appListAfterRename.size should be (1) appListAfterRename.head.logPath should not endWith(EventLoggingListener.IN_PROGRESS) } + test("SPARK-5582: empty log directory") { + val provider = new FsHistoryProvider(createTestConf()) + + val logFile1 = newLogFile("app1", inProgress = true) + writeFile(logFile1, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 1L, "test"), + SparkListenerApplicationEnd(2L)) + + val oldLog = new File(testDir, "old1") + oldLog.mkdir() + + provider.checkForLogs() + val appListAfterRename = provider.getListing() + appListAfterRename.size should be (1) + } + private def writeFile(file: File, isNewFormat: Boolean, codec: Option[CompressionCodec], events: SparkListenerEvent*) = { - val out = - if (isNewFormat) { - EventLoggingListener.initEventLog(new FileOutputStream(file), codec) - } else { - val fileStream = new FileOutputStream(file) - codec.map(_.compressedOutputStream(fileStream)).getOrElse(fileStream) - } - val writer = new OutputStreamWriter(out, "UTF-8") + val fstream = new FileOutputStream(file) + val cstream = codec.map(_.compressedOutputStream(fstream)).getOrElse(fstream) + val bstream = new BufferedOutputStream(cstream) + if (isNewFormat) { + EventLoggingListener.initEventLog(new FileOutputStream(file)) + } + val writer = new OutputStreamWriter(bstream, "UTF-8") try { events.foreach(e => writer.write(compact(render(JsonProtocol.sparkEventToJson(e))) + "\n")) } finally { @@ -211,4 +235,8 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers new FileOutputStream(file).close() } + private def createTestConf(): SparkConf = { + new SparkConf().set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) + } + } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala new file mode 100644 index 0000000000000..3a9963a5ce7b7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -0,0 +1,56 @@ +/* + * 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 javax.servlet.http.HttpServletRequest + +import scala.collection.mutable + +import org.apache.hadoop.fs.Path +import org.mockito.Mockito.{when} +import org.scalatest.FunSuite +import org.scalatest.Matchers +import org.scalatest.mock.MockitoSugar + +import org.apache.spark.ui.SparkUI + +class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar { + + test("generate history page with relative links") { + val historyServer = mock[HistoryServer] + val request = mock[HttpServletRequest] + val ui = mock[SparkUI] + val link = "/history/app1" + val info = new ApplicationHistoryInfo("app1", "app1", 0, 2, 1, "xxx", true) + when(historyServer.getApplicationList()).thenReturn(Seq(info)) + when(ui.basePath).thenReturn(link) + when(historyServer.getProviderConfig()).thenReturn(Map[String, String]()) + val page = new HistoryPage(historyServer) + + //when + val response = page.render(request) + + //then + val links = response \\ "a" + val justHrefs = for { + l <- links + attrs <- l.attribute("href") + } yield (attrs.toString) + justHrefs should contain(link) + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 3d2335f9b3637..34c74d87f0a62 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -20,30 +20,46 @@ package org.apache.spark.deploy.master import akka.actor.Address import org.scalatest.FunSuite -import org.apache.spark.SparkException +import org.apache.spark.{SSLOptions, SparkConf, SparkException} class MasterSuite extends FunSuite { test("toAkkaUrl") { - val akkaUrl = Master.toAkkaUrl("spark://1.2.3.4:1234") + val conf = new SparkConf(loadDefaults = false) + val akkaUrl = Master.toAkkaUrl("spark://1.2.3.4:1234", "akka.tcp") assert("akka.tcp://sparkMaster@1.2.3.4:1234/user/Master" === akkaUrl) } + test("toAkkaUrl with SSL") { + val conf = new SparkConf(loadDefaults = false) + val akkaUrl = Master.toAkkaUrl("spark://1.2.3.4:1234", "akka.ssl.tcp") + assert("akka.ssl.tcp://sparkMaster@1.2.3.4:1234/user/Master" === akkaUrl) + } + test("toAkkaUrl: a typo url") { + val conf = new SparkConf(loadDefaults = false) val e = intercept[SparkException] { - Master.toAkkaUrl("spark://1.2. 3.4:1234") + Master.toAkkaUrl("spark://1.2. 3.4:1234", "akka.tcp") } assert("Invalid master URL: spark://1.2. 3.4:1234" === e.getMessage) } test("toAkkaAddress") { - val address = Master.toAkkaAddress("spark://1.2.3.4:1234") + val conf = new SparkConf(loadDefaults = false) + val address = Master.toAkkaAddress("spark://1.2.3.4:1234", "akka.tcp") assert(Address("akka.tcp", "sparkMaster", "1.2.3.4", 1234) === address) } + test("toAkkaAddress with SSL") { + val conf = new SparkConf(loadDefaults = false) + val address = Master.toAkkaAddress("spark://1.2.3.4:1234", "akka.ssl.tcp") + assert(Address("akka.ssl.tcp", "sparkMaster", "1.2.3.4", 1234) === address) + } + test("toAkkaAddress: a typo url") { + val conf = new SparkConf(loadDefaults = false) val e = intercept[SparkException] { - Master.toAkkaAddress("spark://1.2. 3.4:1234") + Master.toAkkaAddress("spark://1.2. 3.4:1234", "akka.tcp") } assert("Invalid master URL: spark://1.2. 3.4:1234" === e.getMessage) } diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala new file mode 100644 index 0000000000000..2fa90e3bd1c63 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -0,0 +1,606 @@ +/* + * 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.rest + +import java.io.DataOutputStream +import java.net.{HttpURLConnection, URL} +import javax.servlet.http.HttpServletResponse + +import scala.collection.mutable + +import akka.actor.{Actor, ActorRef, ActorSystem, Props} +import com.google.common.base.Charsets +import org.scalatest.{BeforeAndAfterEach, FunSuite} +import org.json4s.JsonAST._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark._ +import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.deploy.DeployMessages._ +import org.apache.spark.deploy.{SparkSubmit, SparkSubmitArguments} +import org.apache.spark.deploy.master.DriverState._ + +/** + * Tests for the REST application submission protocol used in standalone cluster mode. + */ +class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { + private val client = new StandaloneRestClient + private var actorSystem: Option[ActorSystem] = None + private var server: Option[StandaloneRestServer] = None + + override def afterEach() { + actorSystem.foreach(_.shutdown()) + server.foreach(_.stop()) + } + + test("construct submit request") { + val appArgs = Array("one", "two", "three") + val sparkProperties = Map("spark.app.name" -> "pi") + val environmentVariables = Map("SPARK_ONE" -> "UN", "SPARK_TWO" -> "DEUX") + val request = client.constructSubmitRequest( + "my-app-resource", "my-main-class", appArgs, sparkProperties, environmentVariables) + assert(request.action === Utils.getFormattedClassName(request)) + assert(request.clientSparkVersion === SPARK_VERSION) + assert(request.appResource === "my-app-resource") + assert(request.mainClass === "my-main-class") + assert(request.appArgs === appArgs) + assert(request.sparkProperties === sparkProperties) + assert(request.environmentVariables === environmentVariables) + } + + test("create submission") { + val submittedDriverId = "my-driver-id" + val submitMessage = "your driver is submitted" + val masterUrl = startDummyServer(submitId = submittedDriverId, submitMessage = submitMessage) + val appArgs = Array("one", "two", "four") + val request = constructSubmitRequest(masterUrl, appArgs) + assert(request.appArgs === appArgs) + assert(request.sparkProperties("spark.master") === masterUrl) + val response = client.createSubmission(masterUrl, request) + val submitResponse = getSubmitResponse(response) + assert(submitResponse.action === Utils.getFormattedClassName(submitResponse)) + assert(submitResponse.serverSparkVersion === SPARK_VERSION) + assert(submitResponse.message === submitMessage) + assert(submitResponse.submissionId === submittedDriverId) + assert(submitResponse.success) + } + + test("create submission from main method") { + val submittedDriverId = "your-driver-id" + val submitMessage = "my driver is submitted" + val masterUrl = startDummyServer(submitId = submittedDriverId, submitMessage = submitMessage) + val conf = new SparkConf(loadDefaults = false) + conf.set("spark.master", masterUrl) + conf.set("spark.app.name", "dreamer") + val appArgs = Array("one", "two", "six") + // main method calls this + val response = StandaloneRestClient.run("app-resource", "main-class", appArgs, conf) + val submitResponse = getSubmitResponse(response) + assert(submitResponse.action === Utils.getFormattedClassName(submitResponse)) + assert(submitResponse.serverSparkVersion === SPARK_VERSION) + assert(submitResponse.message === submitMessage) + assert(submitResponse.submissionId === submittedDriverId) + assert(submitResponse.success) + } + + test("kill submission") { + val submissionId = "my-lyft-driver" + val killMessage = "your driver is killed" + val masterUrl = startDummyServer(killMessage = killMessage) + val response = client.killSubmission(masterUrl, submissionId) + val killResponse = getKillResponse(response) + assert(killResponse.action === Utils.getFormattedClassName(killResponse)) + assert(killResponse.serverSparkVersion === SPARK_VERSION) + assert(killResponse.message === killMessage) + assert(killResponse.submissionId === submissionId) + assert(killResponse.success) + } + + test("request submission status") { + val submissionId = "my-uber-driver" + val submissionState = KILLED + val submissionException = new Exception("there was an irresponsible mix of alcohol and cars") + val masterUrl = startDummyServer(state = submissionState, exception = Some(submissionException)) + val response = client.requestSubmissionStatus(masterUrl, submissionId) + val statusResponse = getStatusResponse(response) + assert(statusResponse.action === Utils.getFormattedClassName(statusResponse)) + assert(statusResponse.serverSparkVersion === SPARK_VERSION) + assert(statusResponse.message.contains(submissionException.getMessage)) + assert(statusResponse.submissionId === submissionId) + assert(statusResponse.driverState === submissionState.toString) + assert(statusResponse.success) + } + + test("create then kill") { + val masterUrl = startSmartServer() + val request = constructSubmitRequest(masterUrl) + val response1 = client.createSubmission(masterUrl, request) + val submitResponse = getSubmitResponse(response1) + assert(submitResponse.success) + assert(submitResponse.submissionId != null) + // kill submission that was just created + val submissionId = submitResponse.submissionId + val response2 = client.killSubmission(masterUrl, submissionId) + val killResponse = getKillResponse(response2) + assert(killResponse.success) + assert(killResponse.submissionId === submissionId) + } + + test("create then request status") { + val masterUrl = startSmartServer() + val request = constructSubmitRequest(masterUrl) + val response1 = client.createSubmission(masterUrl, request) + val submitResponse = getSubmitResponse(response1) + assert(submitResponse.success) + assert(submitResponse.submissionId != null) + // request status of submission that was just created + val submissionId = submitResponse.submissionId + val response2 = client.requestSubmissionStatus(masterUrl, submissionId) + val statusResponse = getStatusResponse(response2) + assert(statusResponse.success) + assert(statusResponse.submissionId === submissionId) + assert(statusResponse.driverState === RUNNING.toString) + } + + test("create then kill then request status") { + val masterUrl = startSmartServer() + val request = constructSubmitRequest(masterUrl) + val response1 = client.createSubmission(masterUrl, request) + val response2 = client.createSubmission(masterUrl, request) + val submitResponse1 = getSubmitResponse(response1) + val submitResponse2 = getSubmitResponse(response2) + assert(submitResponse1.success) + assert(submitResponse2.success) + assert(submitResponse1.submissionId != null) + assert(submitResponse2.submissionId != null) + val submissionId1 = submitResponse1.submissionId + val submissionId2 = submitResponse2.submissionId + // kill only submission 1, but not submission 2 + val response3 = client.killSubmission(masterUrl, submissionId1) + val killResponse = getKillResponse(response3) + assert(killResponse.success) + assert(killResponse.submissionId === submissionId1) + // request status for both submissions: 1 should be KILLED but 2 should be RUNNING still + val response4 = client.requestSubmissionStatus(masterUrl, submissionId1) + val response5 = client.requestSubmissionStatus(masterUrl, submissionId2) + val statusResponse1 = getStatusResponse(response4) + val statusResponse2 = getStatusResponse(response5) + assert(statusResponse1.submissionId === submissionId1) + assert(statusResponse2.submissionId === submissionId2) + assert(statusResponse1.driverState === KILLED.toString) + assert(statusResponse2.driverState === RUNNING.toString) + } + + test("kill or request status before create") { + val masterUrl = startSmartServer() + val doesNotExist = "does-not-exist" + // kill a non-existent submission + val response1 = client.killSubmission(masterUrl, doesNotExist) + val killResponse = getKillResponse(response1) + assert(!killResponse.success) + assert(killResponse.submissionId === doesNotExist) + // request status for a non-existent submission + val response2 = client.requestSubmissionStatus(masterUrl, doesNotExist) + val statusResponse = getStatusResponse(response2) + assert(!statusResponse.success) + assert(statusResponse.submissionId === doesNotExist) + } + + /* ---------------------------------------- * + | Aberrant client / server behavior | + * ---------------------------------------- */ + + test("good request paths") { + val masterUrl = startSmartServer() + val httpUrl = masterUrl.replace("spark://", "http://") + val v = StandaloneRestServer.PROTOCOL_VERSION + val json = constructSubmitRequest(masterUrl).toJson + val submitRequestPath = s"$httpUrl/$v/submissions/create" + val killRequestPath = s"$httpUrl/$v/submissions/kill" + val statusRequestPath = s"$httpUrl/$v/submissions/status" + val (response1, code1) = sendHttpRequestWithResponse(submitRequestPath, "POST", json) + val (response2, code2) = sendHttpRequestWithResponse(s"$killRequestPath/anything", "POST") + val (response3, code3) = sendHttpRequestWithResponse(s"$killRequestPath/any/thing", "POST") + val (response4, code4) = sendHttpRequestWithResponse(s"$statusRequestPath/anything", "GET") + val (response5, code5) = sendHttpRequestWithResponse(s"$statusRequestPath/any/thing", "GET") + // these should all succeed and the responses should be of the correct types + getSubmitResponse(response1) + val killResponse1 = getKillResponse(response2) + val killResponse2 = getKillResponse(response3) + val statusResponse1 = getStatusResponse(response4) + val statusResponse2 = getStatusResponse(response5) + assert(killResponse1.submissionId === "anything") + assert(killResponse2.submissionId === "any") + assert(statusResponse1.submissionId === "anything") + assert(statusResponse2.submissionId === "any") + assert(code1 === HttpServletResponse.SC_OK) + assert(code2 === HttpServletResponse.SC_OK) + assert(code3 === HttpServletResponse.SC_OK) + assert(code4 === HttpServletResponse.SC_OK) + assert(code5 === HttpServletResponse.SC_OK) + } + + test("good request paths, bad requests") { + val masterUrl = startSmartServer() + val httpUrl = masterUrl.replace("spark://", "http://") + val v = StandaloneRestServer.PROTOCOL_VERSION + val submitRequestPath = s"$httpUrl/$v/submissions/create" + val killRequestPath = s"$httpUrl/$v/submissions/kill" + val statusRequestPath = s"$httpUrl/$v/submissions/status" + val goodJson = constructSubmitRequest(masterUrl).toJson + val badJson1 = goodJson.replaceAll("action", "fraction") // invalid JSON + val badJson2 = goodJson.substring(goodJson.size / 2) // malformed JSON + val notJson = "\"hello, world\"" + val (response1, code1) = sendHttpRequestWithResponse(submitRequestPath, "POST") // missing JSON + val (response2, code2) = sendHttpRequestWithResponse(submitRequestPath, "POST", badJson1) + val (response3, code3) = sendHttpRequestWithResponse(submitRequestPath, "POST", badJson2) + val (response4, code4) = sendHttpRequestWithResponse(killRequestPath, "POST") // missing ID + val (response5, code5) = sendHttpRequestWithResponse(s"$killRequestPath/", "POST") + val (response6, code6) = sendHttpRequestWithResponse(statusRequestPath, "GET") // missing ID + val (response7, code7) = sendHttpRequestWithResponse(s"$statusRequestPath/", "GET") + val (response8, code8) = sendHttpRequestWithResponse(submitRequestPath, "POST", notJson) + // these should all fail as error responses + getErrorResponse(response1) + getErrorResponse(response2) + getErrorResponse(response3) + getErrorResponse(response4) + getErrorResponse(response5) + getErrorResponse(response6) + getErrorResponse(response7) + getErrorResponse(response8) + assert(code1 === HttpServletResponse.SC_BAD_REQUEST) + assert(code2 === HttpServletResponse.SC_BAD_REQUEST) + assert(code3 === HttpServletResponse.SC_BAD_REQUEST) + assert(code4 === HttpServletResponse.SC_BAD_REQUEST) + assert(code5 === HttpServletResponse.SC_BAD_REQUEST) + assert(code6 === HttpServletResponse.SC_BAD_REQUEST) + assert(code7 === HttpServletResponse.SC_BAD_REQUEST) + assert(code8 === HttpServletResponse.SC_BAD_REQUEST) + } + + test("bad request paths") { + val masterUrl = startSmartServer() + val httpUrl = masterUrl.replace("spark://", "http://") + val v = StandaloneRestServer.PROTOCOL_VERSION + val (response1, code1) = sendHttpRequestWithResponse(httpUrl, "GET") + val (response2, code2) = sendHttpRequestWithResponse(s"$httpUrl/", "GET") + val (response3, code3) = sendHttpRequestWithResponse(s"$httpUrl/$v", "GET") + val (response4, code4) = sendHttpRequestWithResponse(s"$httpUrl/$v/", "GET") + val (response5, code5) = sendHttpRequestWithResponse(s"$httpUrl/$v/submissions", "GET") + val (response6, code6) = sendHttpRequestWithResponse(s"$httpUrl/$v/submissions/", "GET") + val (response7, code7) = sendHttpRequestWithResponse(s"$httpUrl/$v/submissions/bad", "GET") + val (response8, code8) = sendHttpRequestWithResponse(s"$httpUrl/bad-version", "GET") + assert(code1 === HttpServletResponse.SC_BAD_REQUEST) + assert(code2 === HttpServletResponse.SC_BAD_REQUEST) + assert(code3 === HttpServletResponse.SC_BAD_REQUEST) + assert(code4 === HttpServletResponse.SC_BAD_REQUEST) + assert(code5 === HttpServletResponse.SC_BAD_REQUEST) + assert(code6 === HttpServletResponse.SC_BAD_REQUEST) + assert(code7 === HttpServletResponse.SC_BAD_REQUEST) + assert(code8 === StandaloneRestServer.SC_UNKNOWN_PROTOCOL_VERSION) + // all responses should be error responses + val errorResponse1 = getErrorResponse(response1) + val errorResponse2 = getErrorResponse(response2) + val errorResponse3 = getErrorResponse(response3) + val errorResponse4 = getErrorResponse(response4) + val errorResponse5 = getErrorResponse(response5) + val errorResponse6 = getErrorResponse(response6) + val errorResponse7 = getErrorResponse(response7) + val errorResponse8 = getErrorResponse(response8) + // only the incompatible version response should have server protocol version set + assert(errorResponse1.highestProtocolVersion === null) + assert(errorResponse2.highestProtocolVersion === null) + assert(errorResponse3.highestProtocolVersion === null) + assert(errorResponse4.highestProtocolVersion === null) + assert(errorResponse5.highestProtocolVersion === null) + assert(errorResponse6.highestProtocolVersion === null) + assert(errorResponse7.highestProtocolVersion === null) + assert(errorResponse8.highestProtocolVersion === StandaloneRestServer.PROTOCOL_VERSION) + } + + test("server returns unknown fields") { + val masterUrl = startSmartServer() + val httpUrl = masterUrl.replace("spark://", "http://") + val v = StandaloneRestServer.PROTOCOL_VERSION + val submitRequestPath = s"$httpUrl/$v/submissions/create" + val oldJson = constructSubmitRequest(masterUrl).toJson + val oldFields = parse(oldJson).asInstanceOf[JObject].obj + val newFields = oldFields ++ Seq( + JField("tomato", JString("not-a-fruit")), + JField("potato", JString("not-po-tah-to")) + ) + val newJson = pretty(render(JObject(newFields))) + // send two requests, one with the unknown fields and the other without + val (response1, code1) = sendHttpRequestWithResponse(submitRequestPath, "POST", oldJson) + val (response2, code2) = sendHttpRequestWithResponse(submitRequestPath, "POST", newJson) + val submitResponse1 = getSubmitResponse(response1) + val submitResponse2 = getSubmitResponse(response2) + assert(code1 === HttpServletResponse.SC_OK) + assert(code2 === HttpServletResponse.SC_OK) + // only the response to the modified request should have unknown fields set + assert(submitResponse1.unknownFields === null) + assert(submitResponse2.unknownFields === Array("tomato", "potato")) + } + + test("client handles faulty server") { + val masterUrl = startFaultyServer() + val httpUrl = masterUrl.replace("spark://", "http://") + val v = StandaloneRestServer.PROTOCOL_VERSION + val submitRequestPath = s"$httpUrl/$v/submissions/create" + val killRequestPath = s"$httpUrl/$v/submissions/kill/anything" + val statusRequestPath = s"$httpUrl/$v/submissions/status/anything" + val json = constructSubmitRequest(masterUrl).toJson + // server returns malformed response unwittingly + // client should throw an appropriate exception to indicate server failure + val conn1 = sendHttpRequest(submitRequestPath, "POST", json) + intercept[SubmitRestProtocolException] { client.readResponse(conn1) } + // server attempts to send invalid response, but fails internally on validation + // client should receive an error response as server is able to recover + val conn2 = sendHttpRequest(killRequestPath, "POST") + val response2 = client.readResponse(conn2) + getErrorResponse(response2) + assert(conn2.getResponseCode === HttpServletResponse.SC_INTERNAL_SERVER_ERROR) + // server explodes internally beyond recovery + // client should throw an appropriate exception to indicate server failure + val conn3 = sendHttpRequest(statusRequestPath, "GET") + intercept[SubmitRestProtocolException] { client.readResponse(conn3) } // empty response + assert(conn3.getResponseCode === HttpServletResponse.SC_INTERNAL_SERVER_ERROR) + } + + /* --------------------- * + | Helper methods | + * --------------------- */ + + /** Start a dummy server that responds to requests using the specified parameters. */ + private def startDummyServer( + submitId: String = "fake-driver-id", + submitMessage: String = "driver is submitted", + killMessage: String = "driver is killed", + state: DriverState = FINISHED, + exception: Option[Exception] = None): String = { + startServer(new DummyMaster(submitId, submitMessage, killMessage, state, exception)) + } + + /** Start a smarter dummy server that keeps track of submitted driver states. */ + private def startSmartServer(): String = { + startServer(new SmarterMaster) + } + + /** Start a dummy server that is faulty in many ways... */ + private def startFaultyServer(): String = { + startServer(new DummyMaster, faulty = true) + } + + /** + * Start a [[StandaloneRestServer]] that communicates with the given actor. + * If `faulty` is true, start an [[FaultyStandaloneRestServer]] instead. + * Return the master URL that corresponds to the address of this server. + */ + private def startServer(makeFakeMaster: => Actor, faulty: Boolean = false): String = { + val name = "test-standalone-rest-protocol" + val conf = new SparkConf + val localhost = Utils.localHostName() + val securityManager = new SecurityManager(conf) + val (_actorSystem, _) = AkkaUtils.createActorSystem(name, localhost, 0, conf, securityManager) + val fakeMasterRef = _actorSystem.actorOf(Props(makeFakeMaster)) + val _server = + if (faulty) { + new FaultyStandaloneRestServer(localhost, 0, fakeMasterRef, "spark://fake:7077", conf) + } else { + new StandaloneRestServer(localhost, 0, fakeMasterRef, "spark://fake:7077", conf) + } + val port = _server.start() + // set these to clean them up after every test + actorSystem = Some(_actorSystem) + server = Some(_server) + s"spark://$localhost:$port" + } + + /** Create a submit request with real parameters using Spark submit. */ + private def constructSubmitRequest( + masterUrl: String, + appArgs: Array[String] = Array.empty): CreateSubmissionRequest = { + val mainClass = "main-class-not-used" + val mainJar = "dummy-jar-not-used.jar" + val commandLineArgs = Array( + "--deploy-mode", "cluster", + "--master", masterUrl, + "--name", mainClass, + "--class", mainClass, + mainJar) ++ appArgs + val args = new SparkSubmitArguments(commandLineArgs) + val (_, _, sparkProperties, _) = SparkSubmit.prepareSubmitEnvironment(args) + client.constructSubmitRequest( + mainJar, mainClass, appArgs, sparkProperties.toMap, Map.empty) + } + + /** Return the response as a submit response, or fail with error otherwise. */ + private def getSubmitResponse(response: SubmitRestProtocolResponse): CreateSubmissionResponse = { + response match { + case s: CreateSubmissionResponse => s + case e: ErrorResponse => fail(s"Server returned error: ${e.message}") + case r => fail(s"Expected submit response. Actual: ${r.toJson}") + } + } + + /** Return the response as a kill response, or fail with error otherwise. */ + private def getKillResponse(response: SubmitRestProtocolResponse): KillSubmissionResponse = { + response match { + case k: KillSubmissionResponse => k + case e: ErrorResponse => fail(s"Server returned error: ${e.message}") + case r => fail(s"Expected kill response. Actual: ${r.toJson}") + } + } + + /** Return the response as a status response, or fail with error otherwise. */ + private def getStatusResponse(response: SubmitRestProtocolResponse): SubmissionStatusResponse = { + response match { + case s: SubmissionStatusResponse => s + case e: ErrorResponse => fail(s"Server returned error: ${e.message}") + case r => fail(s"Expected status response. Actual: ${r.toJson}") + } + } + + /** Return the response as an error response, or fail if the response was not an error. */ + private def getErrorResponse(response: SubmitRestProtocolResponse): ErrorResponse = { + response match { + case e: ErrorResponse => e + case r => fail(s"Expected error response. Actual: ${r.toJson}") + } + } + + /** + * Send an HTTP request to the given URL using the method and the body specified. + * Return the connection object. + */ + private def sendHttpRequest( + url: String, + method: String, + body: String = ""): HttpURLConnection = { + val conn = new URL(url).openConnection().asInstanceOf[HttpURLConnection] + conn.setRequestMethod(method) + if (body.nonEmpty) { + conn.setDoOutput(true) + val out = new DataOutputStream(conn.getOutputStream) + out.write(body.getBytes(Charsets.UTF_8)) + out.close() + } + conn + } + + /** + * Send an HTTP request to the given URL using the method and the body specified. + * Return a 2-tuple of the response message from the server and the response code. + */ + private def sendHttpRequestWithResponse( + url: String, + method: String, + body: String = ""): (SubmitRestProtocolResponse, Int) = { + val conn = sendHttpRequest(url, method, body) + (client.readResponse(conn), conn.getResponseCode) + } +} + +/** + * A mock standalone Master that responds with dummy messages. + * In all responses, the success parameter is always true. + */ +private class DummyMaster( + submitId: String = "fake-driver-id", + submitMessage: String = "submitted", + killMessage: String = "killed", + state: DriverState = FINISHED, + exception: Option[Exception] = None) + extends Actor { + + override def receive = { + case RequestSubmitDriver(driverDesc) => + sender ! SubmitDriverResponse(success = true, Some(submitId), submitMessage) + case RequestKillDriver(driverId) => + sender ! KillDriverResponse(driverId, success = true, killMessage) + case RequestDriverStatus(driverId) => + sender ! DriverStatusResponse(found = true, Some(state), None, None, exception) + } +} + +/** + * A mock standalone Master that keeps track of drivers that have been submitted. + * + * If a driver is submitted, its state is immediately set to RUNNING. + * If an existing driver is killed, its state is immediately set to KILLED. + * If an existing driver's status is requested, its state is returned in the response. + * Submits are always successful while kills and status requests are successful only + * if the driver was submitted in the past. + */ +private class SmarterMaster extends Actor { + private var counter: Int = 0 + private val submittedDrivers = new mutable.HashMap[String, DriverState] + + override def receive = { + case RequestSubmitDriver(driverDesc) => + val driverId = s"driver-$counter" + submittedDrivers(driverId) = RUNNING + counter += 1 + sender ! SubmitDriverResponse(success = true, Some(driverId), "submitted") + + case RequestKillDriver(driverId) => + val success = submittedDrivers.contains(driverId) + if (success) { + submittedDrivers(driverId) = KILLED + } + sender ! KillDriverResponse(driverId, success, "killed") + + case RequestDriverStatus(driverId) => + val found = submittedDrivers.contains(driverId) + val state = submittedDrivers.get(driverId) + sender ! DriverStatusResponse(found, state, None, None, None) + } +} + +/** + * A [[StandaloneRestServer]] that is faulty in many ways. + * + * When handling a submit request, the server returns a malformed JSON. + * When handling a kill request, the server returns an invalid JSON. + * When handling a status request, the server throws an internal exception. + * The purpose of this class is to test that client handles these cases gracefully. + */ +private class FaultyStandaloneRestServer( + host: String, + requestedPort: Int, + masterActor: ActorRef, + masterUrl: String, + masterConf: SparkConf) + extends StandaloneRestServer(host, requestedPort, masterActor, masterUrl, masterConf) { + + protected override val contextToServlet = Map[String, StandaloneRestServlet]( + s"$baseContext/create/*" -> new MalformedSubmitServlet, + s"$baseContext/kill/*" -> new InvalidKillServlet, + s"$baseContext/status/*" -> new ExplodingStatusServlet, + "/*" -> new ErrorServlet + ) + + /** A faulty servlet that produces malformed responses. */ + class MalformedSubmitServlet extends SubmitRequestServlet(masterActor, masterUrl, masterConf) { + protected override def sendResponse( + responseMessage: SubmitRestProtocolResponse, + responseServlet: HttpServletResponse): Unit = { + val badJson = responseMessage.toJson.drop(10).dropRight(20) + responseServlet.getWriter.write(badJson) + } + } + + /** A faulty servlet that produces invalid responses. */ + class InvalidKillServlet extends KillRequestServlet(masterActor, masterConf) { + protected override def handleKill(submissionId: String): KillSubmissionResponse = { + val k = super.handleKill(submissionId) + k.submissionId = null + k + } + } + + /** A faulty status servlet that explodes. */ + class ExplodingStatusServlet extends StatusRequestServlet(masterActor, masterConf) { + private def explode: Int = 1 / 0 + protected override def handleStatus(submissionId: String): SubmissionStatusResponse = { + val s = super.handleStatus(submissionId) + s.workerId = explode.toString + s + } + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala new file mode 100644 index 0000000000000..1d64ec201e647 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala @@ -0,0 +1,324 @@ +/* + * 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.rest + +import java.lang.Boolean +import java.lang.Integer + +import org.json4s.jackson.JsonMethods._ +import org.scalatest.FunSuite + +import org.apache.spark.SparkConf + +/** + * Tests for the REST application submission protocol. + */ +class SubmitRestProtocolSuite extends FunSuite { + + test("validate") { + val request = new DummyRequest + intercept[SubmitRestProtocolException] { request.validate() } // missing everything + request.clientSparkVersion = "1.2.3" + intercept[SubmitRestProtocolException] { request.validate() } // missing name and age + request.name = "something" + intercept[SubmitRestProtocolException] { request.validate() } // missing only age + request.age = 2 + intercept[SubmitRestProtocolException] { request.validate() } // age too low + request.age = 10 + request.validate() // everything is set properly + request.clientSparkVersion = null + intercept[SubmitRestProtocolException] { request.validate() } // missing only Spark version + request.clientSparkVersion = "1.2.3" + request.name = null + intercept[SubmitRestProtocolException] { request.validate() } // missing only name + request.message = "not-setting-name" + intercept[SubmitRestProtocolException] { request.validate() } // still missing name + } + + test("request to and from JSON") { + val request = new DummyRequest + intercept[SubmitRestProtocolException] { request.toJson } // implicit validation + request.clientSparkVersion = "1.2.3" + request.active = true + request.age = 25 + request.name = "jung" + val json = request.toJson + assertJsonEquals(json, dummyRequestJson) + val newRequest = SubmitRestProtocolMessage.fromJson(json, classOf[DummyRequest]) + assert(newRequest.clientSparkVersion === "1.2.3") + assert(newRequest.clientSparkVersion === "1.2.3") + assert(newRequest.active) + assert(newRequest.age === 25) + assert(newRequest.name === "jung") + assert(newRequest.message === null) + } + + test("response to and from JSON") { + val response = new DummyResponse + response.serverSparkVersion = "3.3.4" + response.success = true + val json = response.toJson + assertJsonEquals(json, dummyResponseJson) + val newResponse = SubmitRestProtocolMessage.fromJson(json, classOf[DummyResponse]) + assert(newResponse.serverSparkVersion === "3.3.4") + assert(newResponse.serverSparkVersion === "3.3.4") + assert(newResponse.success) + assert(newResponse.message === null) + } + + test("CreateSubmissionRequest") { + val message = new CreateSubmissionRequest + intercept[SubmitRestProtocolException] { message.validate() } + message.clientSparkVersion = "1.2.3" + message.appResource = "honey-walnut-cherry.jar" + message.mainClass = "org.apache.spark.examples.SparkPie" + val conf = new SparkConf(false) + conf.set("spark.app.name", "SparkPie") + message.sparkProperties = conf.getAll.toMap + message.validate() + // optional fields + conf.set("spark.jars", "mayonnaise.jar,ketchup.jar") + conf.set("spark.files", "fireball.png") + conf.set("spark.driver.memory", "512m") + conf.set("spark.driver.cores", "180") + conf.set("spark.driver.extraJavaOptions", " -Dslices=5 -Dcolor=mostly_red") + conf.set("spark.driver.extraClassPath", "food-coloring.jar") + conf.set("spark.driver.extraLibraryPath", "pickle.jar") + conf.set("spark.driver.supervise", "false") + conf.set("spark.executor.memory", "256m") + conf.set("spark.cores.max", "10000") + message.sparkProperties = conf.getAll.toMap + message.appArgs = Array("two slices", "a hint of cinnamon") + message.environmentVariables = Map("PATH" -> "/dev/null") + message.validate() + // bad fields + var badConf = conf.clone().set("spark.driver.cores", "one hundred feet") + message.sparkProperties = badConf.getAll.toMap + intercept[SubmitRestProtocolException] { message.validate() } + badConf = conf.clone().set("spark.driver.supervise", "nope, never") + message.sparkProperties = badConf.getAll.toMap + intercept[SubmitRestProtocolException] { message.validate() } + badConf = conf.clone().set("spark.cores.max", "two men") + message.sparkProperties = badConf.getAll.toMap + intercept[SubmitRestProtocolException] { message.validate() } + message.sparkProperties = conf.getAll.toMap + // test JSON + val json = message.toJson + assertJsonEquals(json, submitDriverRequestJson) + val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[CreateSubmissionRequest]) + assert(newMessage.clientSparkVersion === "1.2.3") + assert(newMessage.appResource === "honey-walnut-cherry.jar") + assert(newMessage.mainClass === "org.apache.spark.examples.SparkPie") + assert(newMessage.sparkProperties("spark.app.name") === "SparkPie") + assert(newMessage.sparkProperties("spark.jars") === "mayonnaise.jar,ketchup.jar") + assert(newMessage.sparkProperties("spark.files") === "fireball.png") + assert(newMessage.sparkProperties("spark.driver.memory") === "512m") + assert(newMessage.sparkProperties("spark.driver.cores") === "180") + assert(newMessage.sparkProperties("spark.driver.extraJavaOptions") === " -Dslices=5 -Dcolor=mostly_red") + assert(newMessage.sparkProperties("spark.driver.extraClassPath") === "food-coloring.jar") + assert(newMessage.sparkProperties("spark.driver.extraLibraryPath") === "pickle.jar") + assert(newMessage.sparkProperties("spark.driver.supervise") === "false") + assert(newMessage.sparkProperties("spark.executor.memory") === "256m") + assert(newMessage.sparkProperties("spark.cores.max") === "10000") + assert(newMessage.appArgs === message.appArgs) + assert(newMessage.sparkProperties === message.sparkProperties) + assert(newMessage.environmentVariables === message.environmentVariables) + } + + test("CreateSubmissionResponse") { + val message = new CreateSubmissionResponse + intercept[SubmitRestProtocolException] { message.validate() } + message.serverSparkVersion = "1.2.3" + message.submissionId = "driver_123" + message.success = true + message.validate() + // test JSON + val json = message.toJson + assertJsonEquals(json, submitDriverResponseJson) + val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[CreateSubmissionResponse]) + assert(newMessage.serverSparkVersion === "1.2.3") + assert(newMessage.submissionId === "driver_123") + assert(newMessage.success) + } + + test("KillSubmissionResponse") { + val message = new KillSubmissionResponse + intercept[SubmitRestProtocolException] { message.validate() } + message.serverSparkVersion = "1.2.3" + message.submissionId = "driver_123" + message.success = true + message.validate() + // test JSON + val json = message.toJson + assertJsonEquals(json, killDriverResponseJson) + val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[KillSubmissionResponse]) + assert(newMessage.serverSparkVersion === "1.2.3") + assert(newMessage.submissionId === "driver_123") + assert(newMessage.success) + } + + test("SubmissionStatusResponse") { + val message = new SubmissionStatusResponse + intercept[SubmitRestProtocolException] { message.validate() } + message.serverSparkVersion = "1.2.3" + message.submissionId = "driver_123" + message.success = true + message.validate() + // optional fields + message.driverState = "RUNNING" + message.workerId = "worker_123" + message.workerHostPort = "1.2.3.4:7780" + // test JSON + val json = message.toJson + assertJsonEquals(json, driverStatusResponseJson) + val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[SubmissionStatusResponse]) + assert(newMessage.serverSparkVersion === "1.2.3") + assert(newMessage.submissionId === "driver_123") + assert(newMessage.driverState === "RUNNING") + assert(newMessage.success) + assert(newMessage.workerId === "worker_123") + assert(newMessage.workerHostPort === "1.2.3.4:7780") + } + + test("ErrorResponse") { + val message = new ErrorResponse + intercept[SubmitRestProtocolException] { message.validate() } + message.serverSparkVersion = "1.2.3" + message.message = "Field not found in submit request: X" + message.validate() + // test JSON + val json = message.toJson + assertJsonEquals(json, errorJson) + val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[ErrorResponse]) + assert(newMessage.serverSparkVersion === "1.2.3") + assert(newMessage.message === "Field not found in submit request: X") + } + + private val dummyRequestJson = + """ + |{ + | "action" : "DummyRequest", + | "active" : true, + | "age" : 25, + | "clientSparkVersion" : "1.2.3", + | "name" : "jung" + |} + """.stripMargin + + private val dummyResponseJson = + """ + |{ + | "action" : "DummyResponse", + | "serverSparkVersion" : "3.3.4", + | "success": true + |} + """.stripMargin + + private val submitDriverRequestJson = + """ + |{ + | "action" : "CreateSubmissionRequest", + | "appArgs" : [ "two slices", "a hint of cinnamon" ], + | "appResource" : "honey-walnut-cherry.jar", + | "clientSparkVersion" : "1.2.3", + | "environmentVariables" : { + | "PATH" : "/dev/null" + | }, + | "mainClass" : "org.apache.spark.examples.SparkPie", + | "sparkProperties" : { + | "spark.driver.extraLibraryPath" : "pickle.jar", + | "spark.jars" : "mayonnaise.jar,ketchup.jar", + | "spark.driver.supervise" : "false", + | "spark.app.name" : "SparkPie", + | "spark.cores.max" : "10000", + | "spark.driver.memory" : "512m", + | "spark.files" : "fireball.png", + | "spark.driver.cores" : "180", + | "spark.driver.extraJavaOptions" : " -Dslices=5 -Dcolor=mostly_red", + | "spark.executor.memory" : "256m", + | "spark.driver.extraClassPath" : "food-coloring.jar" + | } + |} + """.stripMargin + + private val submitDriverResponseJson = + """ + |{ + | "action" : "CreateSubmissionResponse", + | "serverSparkVersion" : "1.2.3", + | "submissionId" : "driver_123", + | "success" : true + |} + """.stripMargin + + private val killDriverResponseJson = + """ + |{ + | "action" : "KillSubmissionResponse", + | "serverSparkVersion" : "1.2.3", + | "submissionId" : "driver_123", + | "success" : true + |} + """.stripMargin + + private val driverStatusResponseJson = + """ + |{ + | "action" : "SubmissionStatusResponse", + | "driverState" : "RUNNING", + | "serverSparkVersion" : "1.2.3", + | "submissionId" : "driver_123", + | "success" : true, + | "workerHostPort" : "1.2.3.4:7780", + | "workerId" : "worker_123" + |} + """.stripMargin + + private val errorJson = + """ + |{ + | "action" : "ErrorResponse", + | "message" : "Field not found in submit request: X", + | "serverSparkVersion" : "1.2.3" + |} + """.stripMargin + + /** Assert that the contents in the two JSON strings are equal after ignoring whitespace. */ + private def assertJsonEquals(jsonString1: String, jsonString2: String): Unit = { + val trimmedJson1 = jsonString1.trim + val trimmedJson2 = jsonString2.trim + val json1 = compact(render(parse(trimmedJson1))) + val json2 = compact(render(parse(trimmedJson2))) + // Put this on a separate line to avoid printing comparison twice when test fails + val equals = json1 == json2 + assert(equals, "\"[%s]\" did not equal \"[%s]\"".format(trimmedJson1, trimmedJson2)) + } +} + +private class DummyResponse extends SubmitRestProtocolResponse +private class DummyRequest extends SubmitRestProtocolRequest { + var active: Boolean = null + var age: Integer = null + var name: String = null + protected override def doValidate(): Unit = { + super.doValidate() + assertFieldIsSet(name, "name") + assertFieldIsSet(age, "age") + assert(age > 5, "Not old enough!") + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala similarity index 94% rename from core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala rename to core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala index 7915ee75d8778..1c27d83cf876c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package org.apache.spark.deploy +package org.apache.spark.deploy.worker -import org.apache.spark.deploy.worker.CommandUtils +import org.apache.spark.deploy.Command import org.apache.spark.util.Utils - import org.scalatest.{FunSuite, Matchers} class CommandUtilsSuite extends FunSuite with Matchers { diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index b6f4411e0587a..aa6e4874cecde 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -27,6 +27,7 @@ import org.scalatest.FunSuite import org.apache.spark.SparkConf import org.apache.spark.deploy.{Command, DriverDescription} +import org.apache.spark.util.Clock class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { @@ -129,7 +130,7 @@ class DriverRunnerTest extends FunSuite { .thenReturn(-1) // fail 3 .thenReturn(-1) // fail 4 .thenReturn(0) // success - when(clock.currentTimeMillis()) + when(clock.getTimeMillis()) .thenReturn(0).thenReturn(1000) // fail 1 (short) .thenReturn(1000).thenReturn(2000) // fail 2 (short) .thenReturn(2000).thenReturn(10000) // fail 3 (long) diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 6f233d7cf97aa..6fca6321e5a1b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -32,8 +32,8 @@ class ExecutorRunnerTest extends FunSuite { val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(appId), Map(), Seq(), Seq(), Seq()), "appUiUrl") - val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", - new File(sparkHome), new File("ooga"), "blah", new SparkConf, Seq("localDir"), + val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", 123, + "publicAddr", new File(sparkHome), new File("ooga"), "blah", new SparkConf, Seq("localDir"), ExecutorState.RUNNING) val builder = CommandUtils.buildProcessBuilder(appDesc.command, 512, sparkHome, er.substituteVariables) assert(builder.command().last === appId) diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala new file mode 100644 index 0000000000000..84e2fd7ad936d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala @@ -0,0 +1,57 @@ +/* + * 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.worker + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.Command + +import org.scalatest.{Matchers, FunSuite} + +class WorkerSuite extends FunSuite with Matchers { + + def cmd(javaOpts: String*) = Command("", Seq.empty, Map.empty, Seq.empty, Seq.empty, Seq(javaOpts:_*)) + def conf(opts: (String, String)*) = new SparkConf(loadDefaults = false).setAll(opts) + + test("test isUseLocalNodeSSLConfig") { + Worker.isUseLocalNodeSSLConfig(cmd("-Dasdf=dfgh")) shouldBe false + Worker.isUseLocalNodeSSLConfig(cmd("-Dspark.ssl.useNodeLocalConf=true")) shouldBe true + Worker.isUseLocalNodeSSLConfig(cmd("-Dspark.ssl.useNodeLocalConf=false")) shouldBe false + Worker.isUseLocalNodeSSLConfig(cmd("-Dspark.ssl.useNodeLocalConf=")) shouldBe false + } + + test("test maybeUpdateSSLSettings") { + Worker.maybeUpdateSSLSettings( + cmd("-Dasdf=dfgh", "-Dspark.ssl.opt1=x"), + conf("spark.ssl.opt1" -> "y", "spark.ssl.opt2" -> "z")) + .javaOpts should contain theSameElementsInOrderAs Seq( + "-Dasdf=dfgh", "-Dspark.ssl.opt1=x") + + Worker.maybeUpdateSSLSettings( + cmd("-Dspark.ssl.useNodeLocalConf=false", "-Dspark.ssl.opt1=x"), + conf("spark.ssl.opt1" -> "y", "spark.ssl.opt2" -> "z")) + .javaOpts should contain theSameElementsInOrderAs Seq( + "-Dspark.ssl.useNodeLocalConf=false", "-Dspark.ssl.opt1=x") + + Worker.maybeUpdateSSLSettings( + cmd("-Dspark.ssl.useNodeLocalConf=true", "-Dspark.ssl.opt1=x"), + conf("spark.ssl.opt1" -> "y", "spark.ssl.opt2" -> "z")) + .javaOpts should contain theSameElementsAs Seq( + "-Dspark.ssl.useNodeLocalConf=true", "-Dspark.ssl.opt1=y", "-Dspark.ssl.opt2=z") + + } +} diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala new file mode 100644 index 0000000000000..326e203afe136 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import org.scalatest.FunSuite + +class TaskMetricsSuite extends FunSuite { + test("[SPARK-5701] updateShuffleReadMetrics: ShuffleReadMetrics not added when no shuffle deps") { + val taskMetrics = new TaskMetrics() + taskMetrics.updateShuffleReadMetrics() + assert(taskMetrics.shuffleReadMetrics.isEmpty) + } +} diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index 98b0a16ce88ba..2e58c159a2ed8 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.FunSuite import org.apache.hadoop.io.Text -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.util.Utils import org.apache.hadoop.io.compress.{DefaultCodec, CompressionCodecFactory, GzipCodec} @@ -42,7 +42,15 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { private var factory: CompressionCodecFactory = _ override def beforeAll() { - sc = new SparkContext("local", "test") + // Hadoop's FileSystem caching does not use the Configuration as part of its cache key, which + // can cause Filesystem.get(Configuration) to return a cached instance created with a different + // configuration than the one passed to get() (see HADOOP-8490 for more details). This caused + // hard-to-reproduce test failures, since any suites that were run after this one would inherit + // the new value of "fs.local.block.size" (see SPARK-5227 and SPARK-5679). To work around this, + // we disable FileSystem caching in this suite. + val conf = new SparkConf().set("spark.hadoop.fs.file.impl.disable.cache", "true") + + sc = new SparkContext("local", "test", conf) // Set the block size of local file system to test whether files are split right or not. sc.hadoopConfiguration.setLong("fs.local.block.size", 32) diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index 81db66ae17464..78fa98a3b9065 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -21,44 +21,46 @@ import java.io.{File, FileWriter, PrintWriter} import scala.collection.mutable.ArrayBuffer -import org.scalatest.FunSuite - +import org.apache.commons.lang.math.RandomUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.{LongWritable, Text} -import org.apache.hadoop.mapred.{FileSplit => OldFileSplit, InputSplit => OldInputSplit, JobConf, - LineRecordReader => OldLineRecordReader, RecordReader => OldRecordReader, Reporter, - TextInputFormat => OldTextInputFormat} import org.apache.hadoop.mapred.lib.{CombineFileInputFormat => OldCombineFileInputFormat, - CombineFileSplit => OldCombineFileSplit, CombineFileRecordReader => OldCombineFileRecordReader} -import org.apache.hadoop.mapreduce.{InputSplit => NewInputSplit, RecordReader => NewRecordReader, - TaskAttemptContext} + CombineFileRecordReader => OldCombineFileRecordReader, CombineFileSplit => OldCombineFileSplit} +import org.apache.hadoop.mapred.{JobConf, Reporter, FileSplit => OldFileSplit, + InputSplit => OldInputSplit, LineRecordReader => OldLineRecordReader, + RecordReader => OldRecordReader, TextInputFormat => OldTextInputFormat} import org.apache.hadoop.mapreduce.lib.input.{CombineFileInputFormat => NewCombineFileInputFormat, CombineFileRecordReader => NewCombineFileRecordReader, CombineFileSplit => NewCombineFileSplit, FileSplit => NewFileSplit, TextInputFormat => NewTextInputFormat} +import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} +import org.apache.hadoop.mapreduce.{TaskAttemptContext, InputSplit => NewInputSplit, + RecordReader => NewRecordReader} +import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.SharedSparkContext import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.util.Utils -class InputOutputMetricsSuite extends FunSuite with SharedSparkContext { +class InputOutputMetricsSuite extends FunSuite with SharedSparkContext + with BeforeAndAfter { @transient var tmpDir: File = _ @transient var tmpFile: File = _ @transient var tmpFilePath: String = _ + @transient val numRecords: Int = 100000 + @transient val numBuckets: Int = 10 - override def beforeAll() { - super.beforeAll() - + before { tmpDir = Utils.createTempDir() val testTempDir = new File(tmpDir, "test") testTempDir.mkdir() tmpFile = new File(testTempDir, getClass.getSimpleName + ".txt") val pw = new PrintWriter(new FileWriter(tmpFile)) - for (x <- 1 to 1000000) { - pw.println("s") + for (x <- 1 to numRecords) { + pw.println(RandomUtils.nextInt(numBuckets)) } pw.close() @@ -66,8 +68,7 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext { tmpFilePath = "file://" + tmpFile.getAbsolutePath } - override def afterAll() { - super.afterAll() + after { Utils.deleteRecursively(tmpDir) } @@ -155,6 +156,101 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext { assert(bytesRead >= tmpFile.length()) } + test("input metrics on records read - simple") { + val records = runAndReturnRecordsRead { + sc.textFile(tmpFilePath, 4).count() + } + assert(records == numRecords) + } + + test("input metrics on records read - more stages") { + val records = runAndReturnRecordsRead { + sc.textFile(tmpFilePath, 4) + .map(key => (key.length, 1)) + .reduceByKey(_ + _) + .count() + } + assert(records == numRecords) + } + + test("input metrics on records - New Hadoop API") { + val records = runAndReturnRecordsRead { + sc.newAPIHadoopFile(tmpFilePath, classOf[NewTextInputFormat], classOf[LongWritable], + classOf[Text]).count() + } + assert(records == numRecords) + } + + test("input metrics on recordsd read with cache") { + // prime the cache manager + val rdd = sc.textFile(tmpFilePath, 4).cache() + rdd.collect() + + val records = runAndReturnRecordsRead { + rdd.count() + } + + assert(records == numRecords) + } + + test("shuffle records read metrics") { + val recordsRead = runAndReturnShuffleRecordsRead { + sc.textFile(tmpFilePath, 4) + .map(key => (key, 1)) + .groupByKey() + .collect() + } + assert(recordsRead == numRecords) + } + + test("shuffle records written metrics") { + val recordsWritten = runAndReturnShuffleRecordsWritten { + sc.textFile(tmpFilePath, 4) + .map(key => (key, 1)) + .groupByKey() + .collect() + } + assert(recordsWritten == numRecords) + } + + /** + * Tests the metrics from end to end. + * 1) reading a hadoop file + * 2) shuffle and writing to a hadoop file. + * 3) writing to hadoop file. + */ + test("input read/write and shuffle read/write metrics all line up") { + var inputRead = 0L + var outputWritten = 0L + var shuffleRead = 0L + var shuffleWritten = 0L + sc.addSparkListener(new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val metrics = taskEnd.taskMetrics + metrics.inputMetrics.foreach(inputRead += _.recordsRead) + metrics.outputMetrics.foreach(outputWritten += _.recordsWritten) + metrics.shuffleReadMetrics.foreach(shuffleRead += _.recordsRead) + metrics.shuffleWriteMetrics.foreach(shuffleWritten += _.shuffleRecordsWritten) + } + }) + + val tmpFile = new File(tmpDir, getClass.getSimpleName) + + sc.textFile(tmpFilePath, 4) + .map(key => (key, 1)) + .reduceByKey(_+_) + .saveAsTextFile("file://" + tmpFile.getAbsolutePath) + + sc.listenerBus.waitUntilEmpty(500) + assert(inputRead == numRecords) + + // Only supported on newer Hadoop + if (SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback().isDefined) { + assert(outputWritten == numBuckets) + } + assert(shuffleRead == shuffleWritten) + } + test("input metrics with interleaved reads") { val numPartitions = 2 val cartVector = 0 to 9 @@ -193,18 +289,66 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext { assert(cartesianBytes == firstSize * numPartitions + (cartVector.length * secondSize)) } - private def runAndReturnBytesRead(job : => Unit): Long = { - val taskBytesRead = new ArrayBuffer[Long]() + private def runAndReturnBytesRead(job: => Unit): Long = { + runAndReturnMetrics(job, _.taskMetrics.inputMetrics.map(_.bytesRead)) + } + + private def runAndReturnRecordsRead(job: => Unit): Long = { + runAndReturnMetrics(job, _.taskMetrics.inputMetrics.map(_.recordsRead)) + } + + private def runAndReturnRecordsWritten(job: => Unit): Long = { + runAndReturnMetrics(job, _.taskMetrics.outputMetrics.map(_.recordsWritten)) + } + + private def runAndReturnShuffleRecordsRead(job: => Unit): Long = { + runAndReturnMetrics(job, _.taskMetrics.shuffleReadMetrics.map(_.recordsRead)) + } + + private def runAndReturnShuffleRecordsWritten(job: => Unit): Long = { + runAndReturnMetrics(job, _.taskMetrics.shuffleWriteMetrics.map(_.shuffleRecordsWritten)) + } + + private def runAndReturnMetrics(job: => Unit, + collector: (SparkListenerTaskEnd) => Option[Long]): Long = { + val taskMetrics = new ArrayBuffer[Long]() sc.addSparkListener(new SparkListener() { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - taskBytesRead += taskEnd.taskMetrics.inputMetrics.get.bytesRead + collector(taskEnd).foreach(taskMetrics += _) } }) job sc.listenerBus.waitUntilEmpty(500) - taskBytesRead.sum + taskMetrics.sum + } + + test("output metrics on records written") { + // Only supported on newer Hadoop + if (SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback().isDefined) { + val file = new File(tmpDir, getClass.getSimpleName) + val filePath = "file://" + file.getAbsolutePath + + val records = runAndReturnRecordsWritten { + sc.parallelize(1 to numRecords).saveAsTextFile(filePath) + } + assert(records == numRecords) + } + } + + test("output metrics on records written - new Hadoop API") { + // Only supported on newer Hadoop + if (SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback().isDefined) { + val file = new File(tmpDir, getClass.getSimpleName) + val filePath = "file://" + file.getAbsolutePath + + val records = runAndReturnRecordsWritten { + sc.parallelize(1 to numRecords).map(key => (key.toString, key.toString)) + .saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](filePath) + } + assert(records == numRecords) + } } test("output metrics when writing text file") { @@ -318,4 +462,4 @@ class NewCombineTextRecordReaderWrapper( override def getCurrentValue(): Text = delegate.getCurrentValue override def getProgress(): Float = delegate.getProgress override def close(): Unit = delegate.close() -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala index 1a9ce8c607dcd..37e528435aa5d 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala @@ -27,7 +27,7 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { } test("MetricsConfig with default properties") { - val conf = new MetricsConfig(Option("dummy-file")) + val conf = new MetricsConfig(None) conf.initialize() assert(conf.properties.size() === 4) diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index de306533752c1..4cd0f97368ca3 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -33,6 +33,9 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { val expectedHistogramResults = Array(0) assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) + val emptyRDD: RDD[Double] = sc.emptyRDD + assert(emptyRDD.histogram(buckets) === expectedHistogramResults) + assert(emptyRDD.histogram(buckets, true) === expectedHistogramResults) } test("WorksWithOutOfRangeWithOneBucket") { diff --git a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala index 6138d0bbd57f6..0dc59888f7304 100644 --- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala @@ -29,22 +29,42 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { Class.forName("org.apache.derby.jdbc.EmbeddedDriver") val conn = DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb;create=true") try { - val create = conn.createStatement - create.execute(""" - CREATE TABLE FOO( - ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1), - DATA INTEGER - )""") - create.close() - val insert = conn.prepareStatement("INSERT INTO FOO(DATA) VALUES(?)") - (1 to 100).foreach { i => - insert.setInt(1, i * 2) - insert.executeUpdate + + try { + val create = conn.createStatement + create.execute(""" + CREATE TABLE FOO( + ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1), + DATA INTEGER + )""") + create.close() + val insert = conn.prepareStatement("INSERT INTO FOO(DATA) VALUES(?)") + (1 to 100).foreach { i => + insert.setInt(1, i * 2) + insert.executeUpdate + } + insert.close() + } catch { + case e: SQLException if e.getSQLState == "X0Y32" => + // table exists } - insert.close() - } catch { - case e: SQLException if e.getSQLState == "X0Y32" => + + try { + val create = conn.createStatement + create.execute("CREATE TABLE BIGINT_TEST(ID BIGINT NOT NULL, DATA INTEGER)") + create.close() + val insert = conn.prepareStatement("INSERT INTO BIGINT_TEST VALUES(?,?)") + (1 to 100).foreach { i => + insert.setLong(1, 100000000000000000L + 4000000000000000L * i) + insert.setInt(2, i) + insert.executeUpdate + } + insert.close() + } catch { + case e: SQLException if e.getSQLState == "X0Y32" => // table exists + } + } finally { conn.close() } @@ -62,6 +82,18 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { assert(rdd.count === 100) assert(rdd.reduce(_+_) === 10100) } + + test("large id overflow") { + sc = new SparkContext("local", "test") + val rdd = new JdbcRDD( + sc, + () => { DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb") }, + "SELECT DATA FROM BIGINT_TEST WHERE ? <= ID AND ID <= ?", + 1131544775L, 567279358897692673L, 20, + (r: ResultSet) => { r.getInt(1) } ).cache() + assert(rdd.count === 100) + assert(rdd.reduce(_+_) === 5050) + } after { try { diff --git a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala index a40f2ffeffdf9..64b1c24c47168 100644 --- a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala @@ -119,5 +119,33 @@ class SortingSuite extends FunSuite with SharedSparkContext with Matchers with L partitions(1).last should be > partitions(2).head partitions(2).last should be > partitions(3).head } + + test("get a range of elements in a sorted RDD that is on one partition") { + val pairArr = (1 to 1000).map(x => (x, x)).toArray + val sorted = sc.parallelize(pairArr, 10).sortByKey() + val range = sorted.filterByRange(20, 40).collect() + assert((20 to 40).toArray === range.map(_._1)) + } + + test("get a range of elements over multiple partitions in a descendingly sorted RDD") { + val pairArr = (1000 to 1 by -1).map(x => (x, x)).toArray + val sorted = sc.parallelize(pairArr, 10).sortByKey(false) + val range = sorted.filterByRange(200, 800).collect() + assert((800 to 200 by -1).toArray === range.map(_._1)) + } + + test("get a range of elements in an array not partitioned by a range partitioner") { + val pairArr = util.Random.shuffle((1 to 1000).toList).map(x => (x, x)) + val pairs = sc.parallelize(pairArr,10) + val range = pairs.filterByRange(200, 800).collect() + assert((800 to 200 by -1).toArray.sorted === range.map(_._1).sorted) + } + + test("get a range of elements over multiple partitions but not taking up full partitions") { + val pairArr = (1000 to 1 by -1).map(x => (x, x)).toArray + val sorted = sc.parallelize(pairArr, 10).sortByKey(false) + val range = sorted.filterByRange(250, 850).collect() + assert((850 to 250 by -1).toArray === range.map(_._1)) + } } 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 eb116213f69fc..30119ce5d4eec 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -96,6 +96,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 + override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} } /** Length of time to wait while draining listener events. */ @@ -208,7 +209,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { - runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, null, null, null)) + runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, null, createFakeTaskInfo(), null)) } } } @@ -219,7 +220,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, - Map[Long, Any]((accumId, 1)), null, null)) + Map[Long, Any]((accumId, 1)), createFakeTaskInfo(), null)) } } } @@ -386,6 +387,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar override def defaultParallelism() = 2 override def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], blockManagerId: BlockManagerId): Boolean = true + override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} } val noKillScheduler = new DAGScheduler( sc, @@ -476,7 +478,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null, Map[Long, Any](), - null, + createFakeTaskInfo(), null)) assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.contains(1)) @@ -487,7 +489,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1, "ignored"), null, Map[Long, Any](), - null, + createFakeTaskInfo(), null)) // The SparkListener should not receive redundant failure events. assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) @@ -507,14 +509,14 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(newEpoch > oldEpoch) val taskSet = taskSets(0) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, null, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) // should work because it's a non-failed host - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, null, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null)) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, null, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) // should work because it's a new epoch taskSet.tasks(1).epoch = newEpoch - runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, null, null)) + runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) complete(taskSets(1), Seq((Success, 42), (Success, 43))) @@ -735,7 +737,11 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42))) completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assert(Accumulators.originals(accum.id).value === 1) + + val accVal = Accumulators.originals(accum.id).get.get.value + + assert(accVal === 1) + assertDataStructuresEmpty } @@ -766,5 +772,14 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(scheduler.shuffleToMapStage.isEmpty) assert(scheduler.waitingStages.isEmpty) } + + // Nothing in this test should break if the task info's fields are null, but + // OutputCommitCoordinator requires the task info itself to not be null. + private def createFakeTaskInfo(): TaskInfo = { + val info = new TaskInfo(0, 0, 0, 0L, "", "", TaskLocality.ANY, false) + info.finishTime = 1 // to prevent spurious errors in JobProgressListener + info + } + } diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 437d8693c0b1f..992dde66f982f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.io.{File, FileOutputStream, InputStream, IOException} +import java.net.URI import scala.collection.mutable import scala.io.Source @@ -26,7 +27,7 @@ import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext, SPARK_VERSION} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io._ import org.apache.spark.util.{JsonProtocol, Utils} @@ -78,7 +79,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin test("Basic event logging with compression") { CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => - testEventLogging(compressionCodec = Some(codec)) + testEventLogging(compressionCodec = Some(CompressionCodec.getShortName(codec))) } } @@ -88,25 +89,35 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin test("End-to-end event logging with compression") { CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => - testApplicationEventLogging(compressionCodec = Some(codec)) + testApplicationEventLogging(compressionCodec = Some(CompressionCodec.getShortName(codec))) } } test("Log overwriting") { - val log = new FileOutputStream(new File(testDir, "test")) - log.close() - try { - testEventLogging() - assert(false) - } catch { - case e: IOException => - // Expected, since we haven't enabled log overwrite. - } - + val logUri = EventLoggingListener.getLogPath(testDir.getAbsolutePath, "test") + val logPath = new URI(logUri).getPath + // Create file before writing the event log + new FileOutputStream(new File(logPath)).close() + // Expected IOException, since we haven't enabled log overwrite. + intercept[IOException] { testEventLogging() } // Try again, but enable overwriting. testEventLogging(extraConf = Map("spark.eventLog.overwrite" -> "true")) } + test("Event log name") { + // without compression + assert(s"file:/base-dir/app1" === EventLoggingListener.getLogPath("/base-dir", "app1")) + // with compression + assert(s"file:/base-dir/app1.lzf" === + EventLoggingListener.getLogPath("/base-dir", "app1", Some("lzf"))) + // illegal characters in app ID + assert(s"file:/base-dir/a-fine-mind_dollar_bills__1" === + EventLoggingListener.getLogPath("/base-dir", "a fine:mind$dollar{bills}.1")) + // illegal characters in app ID with compression + assert(s"file:/base-dir/a-fine-mind_dollar_bills__1.lz4" === + EventLoggingListener.getLogPath("/base-dir", "a fine:mind$dollar{bills}.1", Some("lz4"))) + } + /* ----------------- * * Actual test logic * * ----------------- */ @@ -140,15 +151,17 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin eventLogger.stop() // Verify file contains exactly the two events logged - val (logData, version) = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), - fileSystem) + val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) try { val lines = readLines(logData) - assert(lines.size === 2) - assert(lines(0).contains("SparkListenerApplicationStart")) - assert(lines(1).contains("SparkListenerApplicationEnd")) - assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === applicationStart) - assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationEnd) + val logStart = SparkListenerLogStart(SPARK_VERSION) + assert(lines.size === 3) + assert(lines(0).contains("SparkListenerLogStart")) + assert(lines(1).contains("SparkListenerApplicationStart")) + assert(lines(2).contains("SparkListenerApplicationEnd")) + assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) + assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationStart) + assert(JsonProtocol.sparkEventFromJson(parse(lines(2))) === applicationEnd) } finally { logData.close() } @@ -163,8 +176,10 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin val sc = new SparkContext("local-cluster[2,2,512]", "test", conf) assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get + val eventLogPath = eventLogger.logPath val expectedLogDir = testDir.toURI().toString() - assert(eventLogger.logPath.startsWith(expectedLogDir + "/")) + assert(eventLogPath === EventLoggingListener.getLogPath( + expectedLogDir, sc.applicationId, compressionCodec.map(CompressionCodec.getShortName))) // Begin listening for events that trigger asserts val eventExistenceListener = new EventExistenceListener(eventLogger) @@ -178,8 +193,8 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin eventExistenceListener.assertAllCallbacksInvoked() // Make sure expected events exist in the log file. - val (logData, version) = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), - fileSystem) + val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) + val logStart = SparkListenerLogStart(SPARK_VERSION) val lines = readLines(logData) val eventSet = mutable.Set( SparkListenerApplicationStart, @@ -204,6 +219,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin } } } + assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) assert(eventSet.isEmpty, "The following events are missing: " + eventSet.toSeq) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala new file mode 100644 index 0000000000000..c8c957856247a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -0,0 +1,213 @@ +/* + * 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.scheduler + +import java.io.File +import java.util.concurrent.TimeoutException + +import org.mockito.Matchers +import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.hadoop.mapred.{TaskAttemptID, JobConf, TaskAttemptContext, OutputCommitter} + +import org.apache.spark._ +import org.apache.spark.rdd.{RDD, FakeOutputCommitter} +import org.apache.spark.util.Utils + +import scala.concurrent.Await +import scala.concurrent.duration._ +import scala.language.postfixOps + +/** + * Unit tests for the output commit coordination functionality. + * + * The unit test makes both the original task and the speculated task + * attempt to commit, where committing is emulated by creating a + * directory. If both tasks create directories then the end result is + * a failure. + * + * Note that there are some aspects of this test that are less than ideal. + * In particular, the test mocks the speculation-dequeuing logic to always + * dequeue a task and consider it as speculated. Immediately after initially + * submitting the tasks and calling reviveOffers(), reviveOffers() is invoked + * again to pick up the speculated task. This may be hacking the original + * behavior in too much of an unrealistic fashion. + * + * Also, the validation is done by checking the number of files in a directory. + * Ideally, an accumulator would be used for this, where we could increment + * the accumulator in the output committer's commitTask() call. If the call to + * commitTask() was called twice erroneously then the test would ideally fail because + * the accumulator would be incremented twice. + * + * The problem with this test implementation is that when both a speculated task and + * its original counterpart complete, only one of the accumulator's increments is + * captured. This results in a paradox where if the OutputCommitCoordinator logic + * was not in SparkHadoopWriter, the tests would still pass because only one of the + * increments would be captured even though the commit in both tasks was executed + * erroneously. + */ +class OutputCommitCoordinatorSuite extends FunSuite with BeforeAndAfter { + + var outputCommitCoordinator: OutputCommitCoordinator = null + var tempDir: File = null + var sc: SparkContext = null + + before { + tempDir = Utils.createTempDir() + val conf = new SparkConf() + .setMaster("local[4]") + .setAppName(classOf[OutputCommitCoordinatorSuite].getSimpleName) + .set("spark.speculation", "true") + sc = new SparkContext(conf) { + override private[spark] def createSparkEnv( + conf: SparkConf, + isLocal: Boolean, + listenerBus: LiveListenerBus): SparkEnv = { + outputCommitCoordinator = spy(new OutputCommitCoordinator(conf)) + // Use Mockito.spy() to maintain the default infrastructure everywhere else. + // This mocking allows us to control the coordinator responses in test cases. + SparkEnv.createDriverEnv(conf, isLocal, listenerBus, Some(outputCommitCoordinator)) + } + } + // Use Mockito.spy() to maintain the default infrastructure everywhere else + val mockTaskScheduler = spy(sc.taskScheduler.asInstanceOf[TaskSchedulerImpl]) + + doAnswer(new Answer[Unit]() { + override def answer(invoke: InvocationOnMock): Unit = { + // Submit the tasks, then force the task scheduler to dequeue the + // speculated task + invoke.callRealMethod() + mockTaskScheduler.backend.reviveOffers() + } + }).when(mockTaskScheduler).submitTasks(Matchers.any()) + + doAnswer(new Answer[TaskSetManager]() { + override def answer(invoke: InvocationOnMock): TaskSetManager = { + val taskSet = invoke.getArguments()(0).asInstanceOf[TaskSet] + new TaskSetManager(mockTaskScheduler, taskSet, 4) { + var hasDequeuedSpeculatedTask = false + override def dequeueSpeculativeTask( + execId: String, + host: String, + locality: TaskLocality.Value): Option[(Int, TaskLocality.Value)] = { + if (!hasDequeuedSpeculatedTask) { + hasDequeuedSpeculatedTask = true + Some(0, TaskLocality.PROCESS_LOCAL) + } else { + None + } + } + } + } + }).when(mockTaskScheduler).createTaskSetManager(Matchers.any(), Matchers.any()) + + sc.taskScheduler = mockTaskScheduler + val dagSchedulerWithMockTaskScheduler = new DAGScheduler(sc, mockTaskScheduler) + sc.taskScheduler.setDAGScheduler(dagSchedulerWithMockTaskScheduler) + sc.dagScheduler = dagSchedulerWithMockTaskScheduler + } + + after { + sc.stop() + tempDir.delete() + outputCommitCoordinator = null + } + + test("Only one of two duplicate commit tasks should commit") { + val rdd = sc.parallelize(Seq(1), 1) + sc.runJob(rdd, OutputCommitFunctions(tempDir.getAbsolutePath).commitSuccessfully _, + 0 until rdd.partitions.size, allowLocal = false) + assert(tempDir.list().size === 1) + } + + test("If commit fails, if task is retried it should not be locked, and will succeed.") { + val rdd = sc.parallelize(Seq(1), 1) + sc.runJob(rdd, OutputCommitFunctions(tempDir.getAbsolutePath).failFirstCommitAttempt _, + 0 until rdd.partitions.size, allowLocal = false) + assert(tempDir.list().size === 1) + } + + test("Job should not complete if all commits are denied") { + // Create a mock OutputCommitCoordinator that denies all attempts to commit + doReturn(false).when(outputCommitCoordinator).handleAskPermissionToCommit( + Matchers.any(), Matchers.any(), Matchers.any()) + val rdd: RDD[Int] = sc.parallelize(Seq(1), 1) + def resultHandler(x: Int, y: Unit): Unit = {} + val futureAction: SimpleFutureAction[Unit] = sc.submitJob[Int, Unit, Unit](rdd, + OutputCommitFunctions(tempDir.getAbsolutePath).commitSuccessfully, + 0 until rdd.partitions.size, resultHandler, () => Unit) + // It's an error if the job completes successfully even though no committer was authorized, + // so throw an exception if the job was allowed to complete. + intercept[TimeoutException] { + Await.result(futureAction, 5 seconds) + } + assert(tempDir.list().size === 0) + } +} + +/** + * Class with methods that can be passed to runJob to test commits with a mock committer. + */ +private case class OutputCommitFunctions(tempDirPath: String) { + + // Mock output committer that simulates a successful commit (after commit is authorized) + private def successfulOutputCommitter = new FakeOutputCommitter { + override def commitTask(context: TaskAttemptContext): Unit = { + Utils.createDirectory(tempDirPath) + } + } + + // Mock output committer that simulates a failed commit (after commit is authorized) + private def failingOutputCommitter = new FakeOutputCommitter { + override def commitTask(taskAttemptContext: TaskAttemptContext) { + throw new RuntimeException + } + } + + def commitSuccessfully(iter: Iterator[Int]): Unit = { + val ctx = TaskContext.get() + runCommitWithProvidedCommitter(ctx, iter, successfulOutputCommitter) + } + + def failFirstCommitAttempt(iter: Iterator[Int]): Unit = { + val ctx = TaskContext.get() + runCommitWithProvidedCommitter(ctx, iter, + if (ctx.attemptNumber == 0) failingOutputCommitter else successfulOutputCommitter) + } + + private def runCommitWithProvidedCommitter( + ctx: TaskContext, + iter: Iterator[Int], + outputCommitter: OutputCommitter): Unit = { + def jobConf = new JobConf { + override def getOutputCommitter(): OutputCommitter = outputCommitter + } + val sparkHadoopWriter = new SparkHadoopWriter(jobConf) { + override def newTaskAttemptContext( + conf: JobConf, + attemptId: TaskAttemptID): TaskAttemptContext = { + mock(classOf[TaskAttemptContext]) + } + } + sparkHadoopWriter.setup(ctx.stageId, ctx.partitionId, ctx.attemptNumber) + sparkHadoopWriter.commit() + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 7e360cc6082ec..601694f57aad0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -61,7 +61,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { try { val replayer = new ReplayListenerBus() replayer.addListener(eventMonster) - replayer.replay(logData, SPARK_VERSION) + replayer.replay(logData, logFilePath.toString) } finally { logData.close() } @@ -115,12 +115,12 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { assert(!eventLog.isDir) // Replay events - val (logData, version) = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem) + val logData = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem) val eventMonster = new EventMonster(conf) try { val replayer = new ReplayListenerBus() replayer.addListener(eventMonster) - replayer.replay(logData, version) + replayer.replay(logData, eventLog.getPath().toString) } finally { logData.close() } @@ -150,11 +150,4 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { override def start() { } } - - private def getCompressionCodec(codecName: String) = { - val conf = new SparkConf - conf.set("spark.io.compression.codec", codecName) - CompressionCodec.createCodec(conf) - } - } 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 0fb1bdd30d975..3a41ee8d4ae0c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -20,26 +20,22 @@ package org.apache.spark.scheduler import java.util.concurrent.Semaphore import scala.collection.mutable +import scala.collection.JavaConversions._ -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} -import org.scalatest.Matchers +import org.scalatest.{FunSuite, Matchers} -import org.apache.spark.{LocalSparkContext, SparkContext} import org.apache.spark.executor.TaskMetrics import org.apache.spark.util.ResetSystemProperties +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext} -class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers with BeforeAndAfter - with BeforeAndAfterAll with ResetSystemProperties { +class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers + with ResetSystemProperties { /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 val jobCompletionTime = 1421191296660L - before { - sc = new SparkContext("local", "SparkListenerSuite") - } - test("basic creation and shutdown of LiveListenerBus") { val counter = new BasicJobCounter val bus = new LiveListenerBus @@ -127,6 +123,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } test("basic creation of StageInfo") { + sc = new SparkContext("local", "SparkListenerSuite") val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) @@ -148,6 +145,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } test("basic creation of StageInfo with shuffle") { + sc = new SparkContext("local", "SparkListenerSuite") val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) @@ -185,6 +183,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } test("StageInfo with fewer tasks than partitions") { + sc = new SparkContext("local", "SparkListenerSuite") val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) @@ -201,6 +200,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } test("local metrics") { + sc = new SparkContext("local", "SparkListenerSuite") val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) sc.addSparkListener(new StatsReportListener) @@ -267,6 +267,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } test("onTaskGettingResult() called when result fetched remotely") { + sc = new SparkContext("local", "SparkListenerSuite") val listener = new SaveTaskEvents sc.addSparkListener(listener) @@ -287,6 +288,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } test("onTaskGettingResult() not called when result sent directly") { + sc = new SparkContext("local", "SparkListenerSuite") val listener = new SaveTaskEvents sc.addSparkListener(listener) @@ -302,6 +304,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } test("onTaskEnd() should be called for all started tasks, even after job has been killed") { + sc = new SparkContext("local", "SparkListenerSuite") val WAIT_TIMEOUT_MILLIS = 10000 val listener = new SaveTaskEvents sc.addSparkListener(listener) @@ -356,6 +359,17 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers assert(jobCounter2.count === 5) } + test("registering listeners via spark.extraListeners") { + val conf = new SparkConf().setMaster("local").setAppName("test") + .set("spark.extraListeners", classOf[ListenerThatAcceptsSparkConf].getName + "," + + classOf[BasicJobCounter].getName) + sc = new SparkContext(conf) + sc.listenerBus.listeners.collect { case x: BasicJobCounter => x}.size should be (1) + sc.listenerBus.listeners.collect { + case x: ListenerThatAcceptsSparkConf => x + }.size should be (1) + } + /** * Assert that the given list of numbers has an average that is greater than zero. */ @@ -363,14 +377,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers assert(m.sum / m.size.toDouble > 0.0, msg) } - /** - * A simple listener that counts the number of jobs observed. - */ - private class BasicJobCounter extends SparkListener { - var count = 0 - override def onJobEnd(job: SparkListenerJobEnd) = count += 1 - } - /** * A simple listener that saves all task infos and task metrics. */ @@ -423,3 +429,19 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } } + +// These classes can't be declared inside of the SparkListenerSuite class because we don't want +// their constructors to contain references to SparkListenerSuite: + +/** + * A simple listener that counts the number of jobs observed. + */ +private class BasicJobCounter extends SparkListener { + var count = 0 + override def onJobEnd(job: SparkListenerJobEnd) = count += 1 +} + +private class ListenerThatAcceptsSparkConf(conf: SparkConf) extends SparkListener { + var count = 0 + override def onJobEnd(job: SparkListenerJobEnd) = count += 1 +} 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 84b9b788237bf..12330d8f63c40 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -27,7 +27,7 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.FakeClock +import org.apache.spark.util.ManualClock class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) extends DAGScheduler(sc) { @@ -164,7 +164,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // Offer a host with NO_PREF as the constraint, @@ -213,7 +213,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execC", "host2")) val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "execB"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // An executor that is not NODE_LOCAL should be rejected. @@ -234,7 +234,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host1"), TaskLocation("host2", "exec2")), Seq() // Last task has no locality prefs ) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1, exec1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) @@ -263,7 +263,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host2", "exec3")), Seq() // Last task has no locality prefs ) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1, exec1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).get.index === 0) @@ -283,7 +283,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host3")), Seq(TaskLocation("host2")) ) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1: first task should be chosen @@ -314,13 +314,14 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("delay scheduling with failed hosts") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"), + ("exec3", "host3")) val taskSet = FakeTask.createTaskSet(3, Seq(TaskLocation("host1")), Seq(TaskLocation("host2")), Seq(TaskLocation("host3")) ) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1: first task should be chosen @@ -352,7 +353,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) @@ -369,7 +370,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted @@ -401,7 +402,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { ("exec1.1", "host1"), ("exec2", "host2")) // affinity to exec1 on host1 - which we will fail. val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "exec1"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, 4, clock) { @@ -485,7 +486,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host1", "execB")), Seq(TaskLocation("host2", "execC")), Seq()) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // Only ANY is valid assert(manager.myLocalityLevels.sameElements(Array(NO_PREF, ANY))) @@ -521,7 +522,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val taskSet = FakeTask.createTaskSet(2, Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host1", "execA"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY))) @@ -610,7 +611,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host2"), TaskLocation("host1")), Seq(), Seq(TaskLocation("host3", "execC"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 0) @@ -636,7 +637,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host2")), Seq(), Seq(TaskLocation("host3"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // node-local tasks are scheduled without delay @@ -649,6 +650,47 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("execA", "host3", NO_PREF).get.index === 2) } + test("SPARK-4939: node-local tasks should be scheduled right after process-local tasks finished") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2")) + val taskSet = FakeTask.createTaskSet(4, + Seq(TaskLocation("host1")), + Seq(TaskLocation("host2")), + Seq(ExecutorCacheTaskLocation("host1", "execA")), + Seq(ExecutorCacheTaskLocation("host2", "execB"))) + val clock = new ManualClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + + // process-local tasks are scheduled first + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 2) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL).get.index === 3) + // node-local tasks are scheduled without delay + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL).get.index === 1) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL) == None) + } + + test("SPARK-4939: no-pref tasks should be scheduled after process-local tasks finished") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2")) + val taskSet = FakeTask.createTaskSet(3, + Seq(), + Seq(ExecutorCacheTaskLocation("host1", "execA")), + Seq(ExecutorCacheTaskLocation("host2", "execB"))) + val clock = new ManualClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + + // process-local tasks are scheduled first + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 1) + assert(manager.resourceOffer("execB", "host2", PROCESS_LOCAL).get.index === 2) + // no-pref tasks are scheduled without delay + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL) == None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) + assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 0) + assert(manager.resourceOffer("execA", "host1", ANY) == None) + } + test("Ensure TaskSetManager is usable after addition of levels") { // Regression test for SPARK-2931 sc = new SparkContext("local", "test") @@ -656,7 +698,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val taskSet = FakeTask.createTaskSet(2, Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host2", "execB.1"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // Only ANY is valid assert(manager.myLocalityLevels.sameElements(Array(ANY))) @@ -690,7 +732,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(HostTaskLocation("host1")), Seq(HostTaskLocation("host2")), Seq(HDFSCacheTaskLocation("host3"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) sched.removeExecutor("execA") diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala index f2ff98eb72daf..afbaa9ade811f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala @@ -17,45 +17,48 @@ package org.apache.spark.scheduler.mesos -import org.apache.spark.executor.MesosExecutorBackend -import org.scalatest.FunSuite -import org.apache.spark.{SparkConf, SparkContext, LocalSparkContext} -import org.apache.spark.scheduler.{SparkListenerExecutorAdded, LiveListenerBus, - TaskDescription, WorkerOffer, TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.cluster.mesos.{MemoryUtils, MesosSchedulerBackend} -import org.apache.mesos.SchedulerDriver -import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, _} -import org.apache.mesos.Protos.Value.Scalar -import org.easymock.{Capture, EasyMock} import java.nio.ByteBuffer -import java.util.Collections import java.util -import org.scalatest.mock.EasyMockSugar +import java.util.Collections import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with EasyMockSugar { +import org.apache.mesos.SchedulerDriver +import org.apache.mesos.Protos._ +import org.apache.mesos.Protos.Value.Scalar +import org.mockito.Mockito._ +import org.mockito.Matchers._ +import org.mockito.{ArgumentCaptor, Matchers} +import org.scalatest.FunSuite +import org.scalatest.mock.MockitoSugar + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext} +import org.apache.spark.executor.MesosExecutorBackend +import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded, + TaskDescription, TaskSchedulerImpl, WorkerOffer} +import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.scheduler.cluster.mesos.{MesosSchedulerBackend, MemoryUtils} + +class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar { test("check spark-class location correctly") { val conf = new SparkConf conf.set("spark.mesos.executor.home" , "/mesos-home") - val listenerBus = EasyMock.createMock(classOf[LiveListenerBus]) - listenerBus.post(SparkListenerExecutorAdded(EasyMock.anyLong, "s1", new ExecutorInfo("host1", 2))) - EasyMock.replay(listenerBus) - - val sc = EasyMock.createMock(classOf[SparkContext]) - EasyMock.expect(sc.getSparkHome()).andReturn(Option("/spark-home")).anyTimes() - EasyMock.expect(sc.conf).andReturn(conf).anyTimes() - EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes() - EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes() - EasyMock.expect(sc.listenerBus).andReturn(listenerBus) - EasyMock.replay(sc) - val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) - EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() - EasyMock.replay(taskScheduler) + val listenerBus = mock[LiveListenerBus] + listenerBus.post( + SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + + val sc = mock[SparkContext] + when(sc.getSparkHome()).thenReturn(Option("/spark-home")) + + when(sc.conf).thenReturn(conf) + when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) + when(sc.executorMemory).thenReturn(100) + when(sc.listenerBus).thenReturn(listenerBus) + val taskScheduler = mock[TaskSchedulerImpl] + when(taskScheduler.CPUS_PER_TASK).thenReturn(2) val mesosSchedulerBackend = new MesosSchedulerBackend(taskScheduler, sc, "master") @@ -84,20 +87,19 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")).setHostname(s"host${id.toString}").build() } - val driver = EasyMock.createMock(classOf[SchedulerDriver]) - val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) + val driver = mock[SchedulerDriver] + val taskScheduler = mock[TaskSchedulerImpl] - val listenerBus = EasyMock.createMock(classOf[LiveListenerBus]) - listenerBus.post(SparkListenerExecutorAdded(EasyMock.anyLong, "s1", new ExecutorInfo("host1", 2))) - EasyMock.replay(listenerBus) + val listenerBus = mock[LiveListenerBus] + listenerBus.post( + SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) - val sc = EasyMock.createMock(classOf[SparkContext]) - EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes() - EasyMock.expect(sc.getSparkHome()).andReturn(Option("/path")).anyTimes() - EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes() - EasyMock.expect(sc.conf).andReturn(new SparkConf).anyTimes() - EasyMock.expect(sc.listenerBus).andReturn(listenerBus) - EasyMock.replay(sc) + val sc = mock[SparkContext] + when(sc.executorMemory).thenReturn(100) + when(sc.getSparkHome()).thenReturn(Option("/path")) + when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) + when(sc.conf).thenReturn(new SparkConf) + when(sc.listenerBus).thenReturn(listenerBus) val minMem = MemoryUtils.calculateTotalMemory(sc).toInt val minCpu = 4 @@ -121,25 +123,29 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea 2 )) val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) - EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(expectedWorkerOffers))).andReturn(Seq(Seq(taskDesc))) - EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() - EasyMock.replay(taskScheduler) + when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) + when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - val capture = new Capture[util.Collection[TaskInfo]] - EasyMock.expect( + val capture = ArgumentCaptor.forClass(classOf[util.Collection[TaskInfo]]) + when( driver.launchTasks( - EasyMock.eq(Collections.singleton(mesosOffers.get(0).getId)), - EasyMock.capture(capture), - EasyMock.anyObject(classOf[Filters]) + Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + capture.capture(), + any(classOf[Filters]) ) - ).andReturn(Status.valueOf(1)).once - EasyMock.expect(driver.declineOffer(mesosOffers.get(1).getId)).andReturn(Status.valueOf(1)).times(1) - EasyMock.expect(driver.declineOffer(mesosOffers.get(2).getId)).andReturn(Status.valueOf(1)).times(1) - EasyMock.replay(driver) + ).thenReturn(Status.valueOf(1)) + when(driver.declineOffer(mesosOffers.get(1).getId)).thenReturn(Status.valueOf(1)) + when(driver.declineOffer(mesosOffers.get(2).getId)).thenReturn(Status.valueOf(1)) backend.resourceOffers(driver, mesosOffers) - EasyMock.verify(driver) + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + capture.capture(), + any(classOf[Filters]) + ) + verify(driver, times(1)).declineOffer(mesosOffers.get(1).getId) + verify(driver, times(1)).declineOffer(mesosOffers.get(2).getId) assert(capture.getValue.size() == 1) val taskInfo = capture.getValue.iterator().next() assert(taskInfo.getName.equals("n1")) @@ -151,15 +157,13 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea // Unwanted resources offered on an existing node. Make sure they are declined val mesosOffers2 = new java.util.ArrayList[Offer] mesosOffers2.add(createOffer(1, minMem, minCpu)) - EasyMock.reset(taskScheduler) - EasyMock.reset(driver) - EasyMock.expect(taskScheduler.resourceOffers(EasyMock.anyObject(classOf[Seq[WorkerOffer]])).andReturn(Seq(Seq()))) - EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() - EasyMock.replay(taskScheduler) - EasyMock.expect(driver.declineOffer(mesosOffers2.get(0).getId)).andReturn(Status.valueOf(1)).times(1) - EasyMock.replay(driver) + reset(taskScheduler) + reset(driver) + when(taskScheduler.resourceOffers(any(classOf[Seq[WorkerOffer]]))).thenReturn(Seq(Seq())) + when(taskScheduler.CPUS_PER_TASK).thenReturn(2) + when(driver.declineOffer(mesosOffers2.get(0).getId)).thenReturn(Status.valueOf(1)) backend.resourceOffers(driver, mesosOffers2) - EasyMock.verify(driver) + verify(driver, times(1)).declineOffer(mesosOffers2.get(0).getId) } } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala index 855f1b6276089..054a4c64897a9 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -29,9 +29,9 @@ class KryoSerializerDistributedSuite extends FunSuite { test("kryo objects are serialised consistently in different processes") { val conf = new SparkConf(false) - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryo.registrator", classOf[AppJarRegistrator].getName) - conf.set("spark.task.maxFailures", "1") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", classOf[AppJarRegistrator].getName) + .set("spark.task.maxFailures", "1") val jar = TestUtils.createJarWithClasses(List(AppJarRegistrator.customClassName)) conf.setJars(List(jar.getPath)) 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 a70f67af2e62e..6198df84fab3d 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -23,9 +23,10 @@ import scala.reflect.ClassTag import com.esotericsoftware.kryo.Kryo import org.scalatest.FunSuite -import org.apache.spark.{SparkConf, SharedSparkContext} +import org.apache.spark.{SharedSparkContext, SparkConf} +import org.apache.spark.scheduler.HighlyCompressedMapStatus import org.apache.spark.serializer.KryoTest._ - +import org.apache.spark.storage.BlockManagerId class KryoSerializerSuite extends FunSuite with SharedSparkContext { conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") @@ -242,6 +243,38 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { ser.newInstance().deserialize[ClassLoaderTestingObject](bytes) } } + + test("registration of HighlyCompressedMapStatus") { + val conf = new SparkConf(false) + conf.set("spark.kryo.registrationRequired", "true") + + // these cases require knowing the internals of RoaringBitmap a little. Blocks span 2^16 + // values, and they use a bitmap (dense) if they have more than 4096 values, and an + // array (sparse) if they use less. So we just create two cases, one sparse and one dense. + // and we use a roaring bitmap for the empty blocks, so we trigger the dense case w/ mostly + // empty blocks + + val ser = new KryoSerializer(conf).newInstance() + val denseBlockSizes = new Array[Long](5000) + val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L) + Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes => + ser.serialize(HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), blockSizes)) + } + } + + test("serialization buffer overflow reporting") { + import org.apache.spark.SparkException + val kryoBufferMaxProperty = "spark.kryoserializer.buffer.max.mb" + + val largeObject = (1 to 1000000).toArray + + val conf = new SparkConf(false) + conf.set(kryoBufferMaxProperty, "1") + + val ser = new KryoSerializer(conf).newInstance() + val thrown = intercept[SparkException](ser.serialize(largeObject)) + assert(thrown.getMessage.contains(kryoBufferMaxProperty)) + } } 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 ffe6f039145ea..3fdbe99b5d02b 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1064,6 +1064,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach var unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks) verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) assert(memoryStore.currentUnrollMemoryForThisThread === 0) + memoryStore.releasePendingUnrollMemoryForThisThread() // Unroll with not enough space. This should succeed after kicking out someBlock1. store.putIterator("someBlock1", smallList.iterator, StorageLevel.MEMORY_ONLY) @@ -1074,6 +1075,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach assert(droppedBlocks.size === 1) assert(droppedBlocks.head._1 === TestBlockId("someBlock1")) droppedBlocks.clear() + memoryStore.releasePendingUnrollMemoryForThisThread() // Unroll huge block with not enough space. Even after ensuring free space of 12000 * 0.4 = // 4800 bytes, there is still not enough room to unroll this block. This returns an iterator. diff --git a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala index bbc7e1357b90d..c21c92b63ad13 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala @@ -31,6 +31,8 @@ class BlockObjectWriterSuite extends FunSuite { new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) writer.write(Long.box(20)) + // Record metrics update on every write + assert(writeMetrics.shuffleRecordsWritten === 1) // Metrics don't update on every write assert(writeMetrics.shuffleBytesWritten == 0) // After 32 writes, metrics should update @@ -39,6 +41,7 @@ class BlockObjectWriterSuite extends FunSuite { writer.write(Long.box(i)) } assert(writeMetrics.shuffleBytesWritten > 0) + assert(writeMetrics.shuffleRecordsWritten === 33) writer.commitAndClose() assert(file.length() == writeMetrics.shuffleBytesWritten) } @@ -51,6 +54,8 @@ class BlockObjectWriterSuite extends FunSuite { new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) writer.write(Long.box(20)) + // Record metrics update on every write + assert(writeMetrics.shuffleRecordsWritten === 1) // Metrics don't update on every write assert(writeMetrics.shuffleBytesWritten == 0) // After 32 writes, metrics should update @@ -59,7 +64,23 @@ class BlockObjectWriterSuite extends FunSuite { writer.write(Long.box(i)) } assert(writeMetrics.shuffleBytesWritten > 0) + assert(writeMetrics.shuffleRecordsWritten === 33) writer.revertPartialWritesAndClose() assert(writeMetrics.shuffleBytesWritten == 0) + assert(writeMetrics.shuffleRecordsWritten == 0) + } + + test("Reopening a closed block writer") { + val file = new File("somefile") + file.deleteOnExit() + val writeMetrics = new ShuffleWriteMetrics() + val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, + new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) + + writer.open() + writer.close() + intercept[IllegalStateException] { + writer.open() + } } } diff --git a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala index 8cf951adb354b..82a82e23eecf2 100644 --- a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.storage import java.io.File import org.apache.spark.util.Utils -import org.scalatest.FunSuite +import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.SparkConf @@ -28,7 +28,11 @@ import org.apache.spark.SparkConf /** * Tests for the spark.local.dir and SPARK_LOCAL_DIRS configuration options. */ -class LocalDirsSuite extends FunSuite { +class LocalDirsSuite extends FunSuite with BeforeAndAfter { + + before { + Utils.clearLocalRootDirs() + } test("Utils.getLocalDir() returns a valid directory, even if some local dirs are missing") { // Regression test for SPARK-2974 diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index e85a436cdba17..6a972381faf14 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -32,12 +32,21 @@ import org.apache.spark.api.java.StorageLevels import org.apache.spark.shuffle.FetchFailedException /** - * Selenium tests for the Spark Web UI. These tests are not run by default - * because they're slow. + * Selenium tests for the Spark Web UI. */ -@DoNotDiscover -class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { - implicit val webDriver: WebDriver = new HtmlUnitDriver +class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll { + + implicit var webDriver: WebDriver = _ + + override def beforeAll(): Unit = { + webDriver = new HtmlUnitDriver + } + + override def afterAll(): Unit = { + if (webDriver != null) { + webDriver.quit() + } + } /** * Create a test SparkContext with the SparkUI enabled. @@ -48,6 +57,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { .setMaster("local") .setAppName("test") .set("spark.ui.enabled", "true") + .set("spark.ui.port", "0") val sc = new SparkContext(conf) assert(sc.ui.isDefined) sc @@ -93,7 +103,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } eventually(timeout(5 seconds), interval(50 milliseconds)) { go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") - find(id("active")).get.text should be("Active Stages (0)") + find(id("active")) should be(None) // Since we hide empty tables find(id("failed")).get.text should be("Failed Stages (1)") } @@ -105,7 +115,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } eventually(timeout(5 seconds), interval(50 milliseconds)) { go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") - find(id("active")).get.text should be("Active Stages (0)") + find(id("active")) should be(None) // Since we hide empty tables // The failure occurs before the stage becomes active, hence we should still show only one // failed stage, not two: find(id("failed")).get.text should be("Failed Stages (1)") @@ -167,13 +177,14 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { test("job progress bars should handle stage / task failures") { withSpark(newSparkContext()) { sc => - val data = sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity) + val data = sc.parallelize(Seq(1, 2, 3), 1).map(identity).groupBy(identity) val shuffleHandle = data.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle // Simulate fetch failures: val mappedData = data.map { x => val taskContext = TaskContext.get - if (taskContext.attemptNumber == 0) { // Cause this stage to fail on its first attempt. + if (taskContext.taskAttemptId() == 1) { + // Cause the post-shuffle stage to fail on its first attempt with a single task failure val env = SparkEnv.get val bmAddress = env.blockManager.blockManagerId val shuffleId = shuffleHandle.shuffleId 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 68074ae32a672..730a4b54f5aa1 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 @@ -88,6 +88,28 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc listener.completedStages.map(_.stageId).toSet should be (Set(50, 49, 48, 47, 46)) } + test("test clearing of stageIdToActiveJobs") { + val conf = new SparkConf() + conf.set("spark.ui.retainedStages", 5.toString) + val listener = new JobProgressListener(conf) + val jobId = 0 + val stageIds = 1 to 50 + // Start a job with 50 stages + listener.onJobStart(createJobStartEvent(jobId, stageIds)) + for (stageId <- stageIds) { + listener.onStageSubmitted(createStageStartEvent(stageId)) + } + listener.stageIdToActiveJobIds.size should be > 0 + + // Complete the stages and job + for (stageId <- stageIds) { + listener.onStageCompleted(createStageEndEvent(stageId, failed = false)) + } + listener.onJobEnd(createJobEndEvent(jobId, false)) + assertActiveJobsStateIsEmpty(listener) + listener.stageIdToActiveJobIds.size should be (0) + } + test("test LRU eviction of jobs") { val conf = new SparkConf() conf.set("spark.ui.retainedStages", 5.toString) @@ -227,6 +249,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskMetrics.setShuffleReadMetrics(Some(shuffleReadMetrics)) taskMetrics.shuffleWriteMetrics = Some(shuffleWriteMetrics) shuffleReadMetrics.incRemoteBytesRead(base + 1) + shuffleReadMetrics.incLocalBytesRead(base + 9) shuffleReadMetrics.incRemoteBlocksFetched(base + 2) shuffleWriteMetrics.incShuffleBytesWritten(base + 3) taskMetrics.setExecutorRunTime(base + 4) @@ -234,7 +257,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskMetrics.incMemoryBytesSpilled(base + 6) val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) taskMetrics.setInputMetrics(Some(inputMetrics)) - inputMetrics.addBytesRead(base + 7) + inputMetrics.incBytesRead(base + 7) val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop) taskMetrics.outputMetrics = Some(outputMetrics) outputMetrics.setBytesWritten(base + 8) @@ -260,8 +283,8 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc var stage0Data = listener.stageIdToData.get((0, 0)).get var stage1Data = listener.stageIdToData.get((1, 0)).get - assert(stage0Data.shuffleReadBytes == 102) - assert(stage1Data.shuffleReadBytes == 201) + assert(stage0Data.shuffleReadTotalBytes == 220) + assert(stage1Data.shuffleReadTotalBytes == 410) assert(stage0Data.shuffleWriteBytes == 106) assert(stage1Data.shuffleWriteBytes == 203) assert(stage0Data.executorRunTime == 108) @@ -290,8 +313,11 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc stage0Data = listener.stageIdToData.get((0, 0)).get stage1Data = listener.stageIdToData.get((1, 0)).get - assert(stage0Data.shuffleReadBytes == 402) - assert(stage1Data.shuffleReadBytes == 602) + // Task 1235 contributed (100+1)+(100+9) = 210 shuffle bytes, and task 1234 contributed + // (300+1)+(300+9) = 610 total shuffle bytes, so the total for the stage is 820. + assert(stage0Data.shuffleReadTotalBytes == 820) + // Task 1236 contributed 410 shuffle bytes, and task 1237 contributed 810 shuffle bytes. + assert(stage1Data.shuffleReadTotalBytes == 1220) assert(stage0Data.shuffleWriteBytes == 406) assert(stage1Data.shuffleWriteBytes == 606) assert(stage0Data.executorRunTime == 408) diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 6bbf72e929dcb..6250d50fb7036 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -17,7 +17,10 @@ package org.apache.spark.util +import java.util.concurrent.TimeoutException + import scala.concurrent.Await +import scala.util.{Failure, Try} import akka.actor._ @@ -26,6 +29,7 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId +import org.apache.spark.SSLSampleConfigs._ /** @@ -47,7 +51,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "true") @@ -60,7 +64,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro conf = conf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( - s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) val timeout = AkkaUtils.lookupTimeout(conf) intercept[akka.actor.ActorNotFound] { slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) @@ -74,7 +78,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val conf = new SparkConf conf.set("spark.authenticate", "false") conf.set("spark.authenticate.secret", "bad") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) val hostname = "localhost" val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, @@ -85,18 +89,18 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "false") badconf.set("spark.authenticate.secret", "good") - val securityManagerBad = new SecurityManager(badconf); + val securityManagerBad = new SecurityManager(badconf) val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = badconf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( - s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) val timeout = AkkaUtils.lookupTimeout(conf) slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) @@ -124,7 +128,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val conf = new SparkConf conf.set("spark.authenticate", "true") conf.set("spark.authenticate.secret", "good") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) val hostname = "localhost" val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, @@ -135,12 +139,12 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val goodconf = new SparkConf goodconf.set("spark.authenticate", "true") goodconf.set("spark.authenticate.secret", "good") - val securityManagerGood = new SecurityManager(goodconf); + val securityManagerGood = new SecurityManager(goodconf) assert(securityManagerGood.isAuthenticationEnabled() === true) @@ -148,7 +152,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro conf = goodconf, securityManager = securityManagerGood) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( - s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) val timeout = AkkaUtils.lookupTimeout(conf) slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) @@ -175,7 +179,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro conf.set("spark.authenticate", "true") conf.set("spark.authenticate.secret", "good") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) val hostname = "localhost" val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, @@ -186,12 +190,12 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "false") badconf.set("spark.authenticate.secret", "bad") - val securityManagerBad = new SecurityManager(badconf); + val securityManagerBad = new SecurityManager(badconf) assert(securityManagerBad.isAuthenticationEnabled() === false) @@ -199,7 +203,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro conf = badconf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( - s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) val timeout = AkkaUtils.lookupTimeout(conf) intercept[akka.actor.ActorNotFound] { slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) @@ -209,4 +213,174 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro slaveSystem.shutdown() } + test("remote fetch ssl on") { + val conf = sparkSSLConfig() + val securityManager = new SecurityManager(conf) + + val hostname = "localhost" + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + conf = conf, securityManager = securityManager) + System.setProperty("spark.hostPort", hostname + ":" + boundPort) + + assert(securityManager.isAuthenticationEnabled() === false) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + + val slaveConf = sparkSSLConfig() + val securityManagerBad = new SecurityManager(slaveConf) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + conf = slaveConf, securityManager = securityManagerBad) + val slaveTracker = new MapOutputTrackerWorker(conf) + val selection = slaveSystem.actorSelection( + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) + val timeout = AkkaUtils.lookupTimeout(conf) + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + + assert(securityManagerBad.isAuthenticationEnabled() === false) + + masterTracker.registerShuffle(10, 1) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) + masterTracker.registerMapOutput(10, 0, + MapStatus(BlockManagerId("a", "hostA", 1000), Array(1000L))) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + // this should succeed since security off + assert(slaveTracker.getServerStatuses(10, 0).toSeq === + Seq((BlockManagerId("a", "hostA", 1000), size1000))) + + actorSystem.shutdown() + slaveSystem.shutdown() + } + + + test("remote fetch ssl on and security enabled") { + val conf = sparkSSLConfig() + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + val securityManager = new SecurityManager(conf) + + val hostname = "localhost" + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + conf = conf, securityManager = securityManager) + System.setProperty("spark.hostPort", hostname + ":" + boundPort) + + assert(securityManager.isAuthenticationEnabled() === true) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + + val slaveConf = sparkSSLConfig() + slaveConf.set("spark.authenticate", "true") + slaveConf.set("spark.authenticate.secret", "good") + val securityManagerBad = new SecurityManager(slaveConf) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + conf = slaveConf, securityManager = securityManagerBad) + val slaveTracker = new MapOutputTrackerWorker(conf) + val selection = slaveSystem.actorSelection( + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) + val timeout = AkkaUtils.lookupTimeout(conf) + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + + assert(securityManagerBad.isAuthenticationEnabled() === true) + + masterTracker.registerShuffle(10, 1) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) + masterTracker.registerMapOutput(10, 0, + MapStatus(BlockManagerId("a", "hostA", 1000), Array(1000L))) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + assert(slaveTracker.getServerStatuses(10, 0).toSeq === + Seq((BlockManagerId("a", "hostA", 1000), size1000))) + + actorSystem.shutdown() + slaveSystem.shutdown() + } + + + test("remote fetch ssl on and security enabled - bad credentials") { + val conf = sparkSSLConfig() + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + val securityManager = new SecurityManager(conf) + + val hostname = "localhost" + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + conf = conf, securityManager = securityManager) + System.setProperty("spark.hostPort", hostname + ":" + boundPort) + + assert(securityManager.isAuthenticationEnabled() === true) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + + val slaveConf = sparkSSLConfig() + slaveConf.set("spark.authenticate", "true") + slaveConf.set("spark.authenticate.secret", "bad") + val securityManagerBad = new SecurityManager(slaveConf) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + conf = slaveConf, securityManager = securityManagerBad) + val slaveTracker = new MapOutputTrackerWorker(conf) + val selection = slaveSystem.actorSelection( + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) + val timeout = AkkaUtils.lookupTimeout(conf) + intercept[akka.actor.ActorNotFound] { + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + } + + actorSystem.shutdown() + slaveSystem.shutdown() + } + + + test("remote fetch ssl on - untrusted server") { + val conf = sparkSSLConfigUntrusted() + val securityManager = new SecurityManager(conf) + + val hostname = "localhost" + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + conf = conf, securityManager = securityManager) + System.setProperty("spark.hostPort", hostname + ":" + boundPort) + + assert(securityManager.isAuthenticationEnabled() === false) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + + val slaveConf = sparkSSLConfig() + val securityManagerBad = new SecurityManager(slaveConf) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + conf = slaveConf, securityManager = securityManagerBad) + val slaveTracker = new MapOutputTrackerWorker(conf) + val selection = slaveSystem.actorSelection( + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) + val timeout = AkkaUtils.lookupTimeout(conf) + val result = Try(Await.result(selection.resolveOne(timeout * 2), timeout)) + + result match { + case Failure(ex: ActorNotFound) => + case Failure(ex: TimeoutException) => + case r => fail(s"$r is neither Failure(ActorNotFound) nor Failure(TimeoutException)") + } + + actorSystem.shutdown() + slaveSystem.shutdown() + } + } 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 6577ebaa2e9a8..a2be724254d7c 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -76,8 +76,9 @@ class JsonProtocolSuite extends FunSuite { val unpersistRdd = SparkListenerUnpersistRDD(12345) val applicationStart = SparkListenerApplicationStart("The winner of all", None, 42L, "Garfield") val applicationEnd = SparkListenerApplicationEnd(42L) + val logUrlMap = Map("stderr" -> "mystderr", "stdout" -> "mystdout").toMap val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1", - new ExecutorInfo("Hostee.awesome.com", 11)) + new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap)) val executorRemoved = SparkListenerExecutorRemoved(executorRemovedTime, "exec2", "test reason") testEvent(stageSubmitted, stageSubmittedJsonString) @@ -100,13 +101,14 @@ class JsonProtocolSuite extends FunSuite { } test("Dependent Classes") { + val logUrlMap = Map("stderr" -> "mystderr", "stdout" -> "mystdout").toMap 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, hasHadoopInput = false, hasOutput = false)) testBlockManagerId(BlockManagerId("Hong", "Kong", 500)) - testExecutorInfo(new ExecutorInfo("host", 43)) + testExecutorInfo(new ExecutorInfo("host", 43, logUrlMap)) // StorageLevel testStorageLevel(StorageLevel.NONE) @@ -187,6 +189,34 @@ class JsonProtocolSuite extends FunSuite { assert(newMetrics.inputMetrics.isEmpty) } + test("Input/Output records backwards compatibility") { + // records read were added after 1.2 + val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6, + hasHadoopInput = true, hasOutput = true, hasRecords = false) + assert(metrics.inputMetrics.nonEmpty) + assert(metrics.outputMetrics.nonEmpty) + val newJson = JsonProtocol.taskMetricsToJson(metrics) + val oldJson = newJson.removeField { case (field, _) => field == "Records Read" } + .removeField { case (field, _) => field == "Records Written" } + val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson) + assert(newMetrics.inputMetrics.get.recordsRead == 0) + assert(newMetrics.outputMetrics.get.recordsWritten == 0) + } + + test("Shuffle Read/Write records backwards compatibility") { + // records read were added after 1.2 + val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6, + hasHadoopInput = false, hasOutput = false, hasRecords = false) + assert(metrics.shuffleReadMetrics.nonEmpty) + assert(metrics.shuffleWriteMetrics.nonEmpty) + val newJson = JsonProtocol.taskMetricsToJson(metrics) + val oldJson = newJson.removeField { case (field, _) => field == "Total Records Read" } + .removeField { case (field, _) => field == "Shuffle Records Written" } + val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson) + assert(newMetrics.shuffleReadMetrics.get.recordsRead == 0) + assert(newMetrics.shuffleWriteMetrics.get.shuffleRecordsWritten == 0) + } + test("OutputMetrics backward compatibility") { // OutputMetrics were added after 1.1 val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6, hasHadoopInput = false, hasOutput = true) @@ -230,6 +260,18 @@ class JsonProtocolSuite extends FunSuite { assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent)) } + test("ShuffleReadMetrics: Local bytes read and time taken backwards compatibility") { + // Metrics about local shuffle bytes read and local read time were added in 1.3.1. + val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6, + hasHadoopInput = false, hasOutput = false, hasRecords = false) + assert(metrics.shuffleReadMetrics.nonEmpty) + val newJson = JsonProtocol.taskMetricsToJson(metrics) + val oldJson = newJson.removeField { case (field, _) => field == "Local Bytes Read" } + .removeField { case (field, _) => field == "Local Read Time" } + val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson) + assert(newMetrics.shuffleReadMetrics.get.localBytesRead == 0) + } + test("SparkListenerApplicationStart backwards compatibility") { // SparkListenerApplicationStart in Spark 1.0.0 do not have an "appId" property. val applicationStart = SparkListenerApplicationStart("test", None, 1L, "user") @@ -642,7 +684,8 @@ class JsonProtocolSuite extends FunSuite { e: Int, f: Int, hasHadoopInput: Boolean, - hasOutput: Boolean) = { + hasOutput: Boolean, + hasRecords: Boolean = true) = { val t = new TaskMetrics t.setHostname("localhost") t.setExecutorDeserializeTime(a) @@ -654,7 +697,8 @@ class JsonProtocolSuite extends FunSuite { if (hasHadoopInput) { val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) - inputMetrics.addBytesRead(d + e + f) + inputMetrics.incBytesRead(d + e + f) + inputMetrics.incRecordsRead(if (hasRecords) (d + e + f) / 100 else -1) t.setInputMetrics(Some(inputMetrics)) } else { val sr = new ShuffleReadMetrics @@ -662,16 +706,20 @@ class JsonProtocolSuite extends FunSuite { sr.incLocalBlocksFetched(e) sr.incFetchWaitTime(a + d) sr.incRemoteBlocksFetched(f) + sr.incRecordsRead(if (hasRecords) (b + d) / 100 else -1) + sr.incLocalBytesRead(a + f) t.setShuffleReadMetrics(Some(sr)) } if (hasOutput) { val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop) outputMetrics.setBytesWritten(a + b + c) + outputMetrics.setRecordsWritten(if (hasRecords) (a + b + c)/100 else -1) t.outputMetrics = Some(outputMetrics) } else { val sw = new ShuffleWriteMetrics sw.incShuffleBytesWritten(a + b + c) sw.incShuffleWriteTime(b + c + d) + sw.setShuffleRecordsWritten(if (hasRecords) (a + b + c) / 100 else -1) t.shuffleWriteMetrics = Some(sw) } // Make at most 6 blocks @@ -905,11 +953,14 @@ class JsonProtocolSuite extends FunSuite { | "Remote Blocks Fetched": 800, | "Local Blocks Fetched": 700, | "Fetch Wait Time": 900, - | "Remote Bytes Read": 1000 + | "Remote Bytes Read": 1000, + | "Local Bytes Read": 1100, + | "Total Records Read" : 10 | }, | "Shuffle Write Metrics": { | "Shuffle Bytes Written": 1200, - | "Shuffle Write Time": 1500 + | "Shuffle Write Time": 1500, + | "Shuffle Records Written": 12 | }, | "Updated Blocks": [ | { @@ -986,11 +1037,13 @@ class JsonProtocolSuite extends FunSuite { | "Disk Bytes Spilled": 0, | "Shuffle Write Metrics": { | "Shuffle Bytes Written": 1200, - | "Shuffle Write Time": 1500 + | "Shuffle Write Time": 1500, + | "Shuffle Records Written": 12 | }, | "Input Metrics": { | "Data Read Method": "Hadoop", - | "Bytes Read": 2100 + | "Bytes Read": 2100, + | "Records Read": 21 | }, | "Updated Blocks": [ | { @@ -1067,11 +1120,13 @@ class JsonProtocolSuite extends FunSuite { | "Disk Bytes Spilled": 0, | "Input Metrics": { | "Data Read Method": "Hadoop", - | "Bytes Read": 2100 + | "Bytes Read": 2100, + | "Records Read": 21 | }, | "Output Metrics": { | "Data Write Method": "Hadoop", - | "Bytes Written": 1200 + | "Bytes Written": 1200, + | "Records Written": 12 | }, | "Updated Blocks": [ | { @@ -1463,7 +1518,11 @@ class JsonProtocolSuite extends FunSuite { | "Executor ID": "exec1", | "Executor Info": { | "Host": "Hostee.awesome.com", - | "Total Cores": 11 + | "Total Cores": 11, + | "Log Urls" : { + | "stderr" : "mystderr", + | "stdout" : "mystdout" + | } | } |} """ diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala similarity index 73% rename from core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala rename to core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala index e2050e95a1b88..87de90bb0dfb0 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala @@ -15,41 +15,48 @@ * limitations under the License. */ -package org.apache.spark.executor +package org.apache.spark.util import java.net.URLClassLoader import org.scalatest.FunSuite -import org.apache.spark.{LocalSparkContext, SparkContext, SparkException, TestUtils} -import org.apache.spark.util.Utils +import org.apache.spark.{SparkContext, SparkException, TestUtils} -class ExecutorURLClassLoaderSuite extends FunSuite { +class MutableURLClassLoaderSuite extends FunSuite { - val childClassNames = List("FakeClass1", "FakeClass2") - val parentClassNames = List("FakeClass1", "FakeClass2", "FakeClass3") - val urls = List(TestUtils.createJarWithClasses(childClassNames, "1")).toArray - val urls2 = List(TestUtils.createJarWithClasses(parentClassNames, "2")).toArray + val urls2 = List(TestUtils.createJarWithClasses( + classNames = Seq("FakeClass1", "FakeClass2", "FakeClass3"), + toStringValue = "2")).toArray + val urls = List(TestUtils.createJarWithClasses( + classNames = Seq("FakeClass1"), + classNamesWithBase = Seq(("FakeClass2", "FakeClass3")), // FakeClass3 is in parent + toStringValue = "1", + classpathUrls = urls2)).toArray test("child first") { val parentLoader = new URLClassLoader(urls2, null) - val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + val classLoader = new ChildFirstURLClassLoader(urls, parentLoader) val fakeClass = classLoader.loadClass("FakeClass2").newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "1") + val fakeClass2 = classLoader.loadClass("FakeClass2").newInstance() + assert(fakeClass.getClass === fakeClass2.getClass) } test("parent first") { val parentLoader = new URLClassLoader(urls2, null) - val classLoader = new ExecutorURLClassLoader(urls, parentLoader) + val classLoader = new MutableURLClassLoader(urls, parentLoader) val fakeClass = classLoader.loadClass("FakeClass1").newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "2") + val fakeClass2 = classLoader.loadClass("FakeClass1").newInstance() + assert(fakeClass.getClass === fakeClass2.getClass) } test("child first can fall back") { val parentLoader = new URLClassLoader(urls2, null) - val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + val classLoader = new ChildFirstURLClassLoader(urls, parentLoader) val fakeClass = classLoader.loadClass("FakeClass3").newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "2") @@ -57,7 +64,7 @@ class ExecutorURLClassLoaderSuite extends FunSuite { test("child first can fail") { val parentLoader = new URLClassLoader(urls2, null) - val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + val classLoader = new ChildFirstURLClassLoader(urls, parentLoader) intercept[java.lang.ClassNotFoundException] { classLoader.loadClass("FakeClassDoesNotExist").newInstance() } diff --git a/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala index d4b92f33dd9e6..bad1aa99952cf 100644 --- a/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala +++ b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala @@ -19,6 +19,7 @@ package org.apache.spark.util import java.util.Properties +import org.apache.commons.lang3.SerializationUtils import org.scalatest.{BeforeAndAfterEach, Suite} /** @@ -42,7 +43,11 @@ private[spark] trait ResetSystemProperties extends BeforeAndAfterEach { this: Su var oldProperties: Properties = null override def beforeEach(): Unit = { - oldProperties = new Properties(System.getProperties) + // we need SerializationUtils.clone instead of `new Properties(System.getProperties()` because + // the later way of creating a copy does not copy the properties but it initializes a new + // Properties object with the given properties as defaults. They are not recognized at all + // by standard Scala wrapper over Java Properties then. + oldProperties = SerializationUtils.clone(System.getProperties) super.beforeEach() } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 4544382094f96..b91428efadfd0 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -29,6 +29,9 @@ import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.scalatest.FunSuite +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + import org.apache.spark.SparkConf class UtilsSuite extends FunSuite with ResetSystemProperties { @@ -205,18 +208,18 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { child1.setLastModified(System.currentTimeMillis() - (1000 * 30)) // although child1 is old, child2 is still new so return true - assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) child2.setLastModified(System.currentTimeMillis - (1000 * 30)) - assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) parent.setLastModified(System.currentTimeMillis - (1000 * 30)) // although parent and its immediate children are new, child3 is still old // we expect a full recursive search for new files. - assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) child3.setLastModified(System.currentTimeMillis - (1000 * 30)) - assert(!Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + assert(!Utils.doesDirectoryContainAnyNewFiles(parent, 5)) } test("resolveURI") { @@ -336,21 +339,21 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { assert(!tempDir1.exists()) val tempDir2 = Utils.createTempDir() - val tempFile1 = new File(tempDir2, "foo.txt") - Files.touch(tempFile1) - assert(tempFile1.exists()) - Utils.deleteRecursively(tempFile1) - assert(!tempFile1.exists()) + val sourceFile1 = new File(tempDir2, "foo.txt") + Files.touch(sourceFile1) + assert(sourceFile1.exists()) + Utils.deleteRecursively(sourceFile1) + assert(!sourceFile1.exists()) val tempDir3 = new File(tempDir2, "subdir") assert(tempDir3.mkdir()) - val tempFile2 = new File(tempDir3, "bar.txt") - Files.touch(tempFile2) - assert(tempFile2.exists()) + val sourceFile2 = new File(tempDir3, "bar.txt") + Files.touch(sourceFile2) + assert(sourceFile2.exists()) Utils.deleteRecursively(tempDir2) assert(!tempDir2.exists()) assert(!tempDir3.exists()) - assert(!tempFile2.exists()) + assert(!sourceFile2.exists()) } test("loading properties from file") { @@ -381,4 +384,42 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { require(cnt === 2, "prepare should be called twice") require(time < 500, "preparation time should not count") } + + test("fetch hcfs dir") { + val tempDir = Utils.createTempDir() + val sourceDir = new File(tempDir, "source-dir") + val innerSourceDir = Utils.createTempDir(root=sourceDir.getPath) + val sourceFile = File.createTempFile("someprefix", "somesuffix", innerSourceDir) + val targetDir = new File(tempDir, "target-dir") + Files.write("some text", sourceFile, UTF_8) + + val path = new Path("file://" + sourceDir.getAbsolutePath) + val conf = new Configuration() + val fs = Utils.getHadoopFileSystem(path.toString, conf) + + assert(!targetDir.isDirectory()) + Utils.fetchHcfsFile(path, targetDir, fs, new SparkConf(), conf, false) + assert(targetDir.isDirectory()) + + // Copy again to make sure it doesn't error if the dir already exists. + Utils.fetchHcfsFile(path, targetDir, fs, new SparkConf(), conf, false) + + val destDir = new File(targetDir, sourceDir.getName()) + assert(destDir.isDirectory()) + + val destInnerDir = new File(destDir, innerSourceDir.getName) + assert(destInnerDir.isDirectory()) + + val destInnerFile = new File(destInnerDir, sourceFile.getName) + assert(destInnerFile.isFile()) + + val filePath = new Path("file://" + sourceFile.getAbsolutePath) + val testFileDir = new File(tempDir, "test-filename") + val testFileName = "testFName" + val testFilefs = Utils.getHadoopFileSystem(filePath.toString, conf) + Utils.fetchHcfsFile(filePath, testFileDir, testFilefs, new SparkConf(), + conf, false, Some(testFileName)) + val newFileName = new File(testFileDir, testFileName) + assert(newFileName.isFile()) + } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala index 0cb1ed7397655..e0d6cc16bde05 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala @@ -65,6 +65,13 @@ class SorterSuite extends FunSuite { } } + // http://www.envisage-project.eu/timsort-specification-and-verification/ + test("SPARK-5984 TimSort bug") { + val data = TestTimSort.getTimSortBugTestSet(67108864) + new Sorter(new IntArraySortDataFormat).sort(data, 0, data.length, Ordering.Int) + (0 to data.length - 2).foreach(i => assert(data(i) <= data(i + 1))) + } + /** Runs an experiment several times. */ def runExperiment(name: String, skip: Boolean = false)(f: => Unit, prepare: () => Unit): Unit = { if (skip) { diff --git a/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala b/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala index 4918e2d92beb4..daa795a043495 100644 --- a/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala +++ b/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala @@ -44,13 +44,21 @@ class ImplicitSuite { } def testRddToSequenceFileRDDFunctions(): Unit = { - // TODO eliminating `import intToIntWritable` needs refactoring SequenceFileRDDFunctions. - // That will be a breaking change. - import org.apache.spark.SparkContext.intToIntWritable val rdd: org.apache.spark.rdd.RDD[(Int, Int)] = mockRDD rdd.saveAsSequenceFile("/a/test/path") } + def testRddToSequenceFileRDDFunctionsWithWritable(): Unit = { + val rdd: org.apache.spark.rdd.RDD[(org.apache.hadoop.io.IntWritable, org.apache.hadoop.io.Text)] + = mockRDD + rdd.saveAsSequenceFile("/a/test/path") + } + + def testRddToSequenceFileRDDFunctionsWithBytesArray(): Unit = { + val rdd: org.apache.spark.rdd.RDD[(Int, Array[Byte])] = mockRDD + rdd.saveAsSequenceFile("/a/test/path") + } + def testRddToOrderedRDDFunctions(): Unit = { val rdd: org.apache.spark.rdd.RDD[(Int, Int)] = mockRDD rdd.sortByKey() diff --git a/data/mllib/als/sample_movielens_movies.txt b/data/mllib/als/sample_movielens_movies.txt new file mode 100644 index 0000000000000..934a0253849e1 --- /dev/null +++ b/data/mllib/als/sample_movielens_movies.txt @@ -0,0 +1,100 @@ +0::Movie 0::Romance|Comedy +1::Movie 1::Action|Anime +2::Movie 2::Romance|Thriller +3::Movie 3::Action|Romance +4::Movie 4::Anime|Comedy +5::Movie 5::Action|Action +6::Movie 6::Action|Comedy +7::Movie 7::Anime|Comedy +8::Movie 8::Comedy|Action +9::Movie 9::Anime|Thriller +10::Movie 10::Action|Anime +11::Movie 11::Action|Anime +12::Movie 12::Anime|Comedy +13::Movie 13::Thriller|Action +14::Movie 14::Anime|Comedy +15::Movie 15::Comedy|Thriller +16::Movie 16::Anime|Romance +17::Movie 17::Thriller|Action +18::Movie 18::Action|Comedy +19::Movie 19::Anime|Romance +20::Movie 20::Action|Anime +21::Movie 21::Romance|Thriller +22::Movie 22::Romance|Romance +23::Movie 23::Comedy|Comedy +24::Movie 24::Anime|Action +25::Movie 25::Comedy|Comedy +26::Movie 26::Anime|Romance +27::Movie 27::Anime|Anime +28::Movie 28::Thriller|Anime +29::Movie 29::Anime|Romance +30::Movie 30::Thriller|Romance +31::Movie 31::Thriller|Romance +32::Movie 32::Comedy|Anime +33::Movie 33::Comedy|Comedy +34::Movie 34::Anime|Anime +35::Movie 35::Action|Thriller +36::Movie 36::Anime|Romance +37::Movie 37::Romance|Anime +38::Movie 38::Thriller|Romance +39::Movie 39::Romance|Comedy +40::Movie 40::Action|Anime +41::Movie 41::Comedy|Thriller +42::Movie 42::Comedy|Action +43::Movie 43::Thriller|Anime +44::Movie 44::Anime|Action +45::Movie 45::Comedy|Romance +46::Movie 46::Comedy|Action +47::Movie 47::Romance|Comedy +48::Movie 48::Action|Comedy +49::Movie 49::Romance|Romance +50::Movie 50::Comedy|Romance +51::Movie 51::Action|Action +52::Movie 52::Thriller|Action +53::Movie 53::Action|Action +54::Movie 54::Romance|Thriller +55::Movie 55::Anime|Romance +56::Movie 56::Comedy|Action +57::Movie 57::Action|Anime +58::Movie 58::Thriller|Romance +59::Movie 59::Thriller|Comedy +60::Movie 60::Anime|Comedy +61::Movie 61::Comedy|Action +62::Movie 62::Comedy|Romance +63::Movie 63::Romance|Thriller +64::Movie 64::Romance|Action +65::Movie 65::Anime|Romance +66::Movie 66::Comedy|Action +67::Movie 67::Thriller|Anime +68::Movie 68::Thriller|Romance +69::Movie 69::Action|Comedy +70::Movie 70::Thriller|Thriller +71::Movie 71::Action|Comedy +72::Movie 72::Thriller|Romance +73::Movie 73::Comedy|Action +74::Movie 74::Action|Action +75::Movie 75::Action|Action +76::Movie 76::Comedy|Comedy +77::Movie 77::Comedy|Comedy +78::Movie 78::Comedy|Comedy +79::Movie 79::Thriller|Thriller +80::Movie 80::Comedy|Anime +81::Movie 81::Comedy|Anime +82::Movie 82::Romance|Anime +83::Movie 83::Comedy|Thriller +84::Movie 84::Anime|Action +85::Movie 85::Thriller|Anime +86::Movie 86::Romance|Anime +87::Movie 87::Thriller|Thriller +88::Movie 88::Romance|Thriller +89::Movie 89::Action|Anime +90::Movie 90::Anime|Romance +91::Movie 91::Anime|Thriller +92::Movie 92::Action|Comedy +93::Movie 93::Romance|Thriller +94::Movie 94::Thriller|Comedy +95::Movie 95::Action|Action +96::Movie 96::Thriller|Romance +97::Movie 97::Thriller|Thriller +98::Movie 98::Thriller|Comedy +99::Movie 99::Thriller|Romance diff --git a/data/mllib/als/sample_movielens_ratings.txt b/data/mllib/als/sample_movielens_ratings.txt new file mode 100644 index 0000000000000..0889142950797 --- /dev/null +++ b/data/mllib/als/sample_movielens_ratings.txt @@ -0,0 +1,1501 @@ +0::2::3::1424380312 +0::3::1::1424380312 +0::5::2::1424380312 +0::9::4::1424380312 +0::11::1::1424380312 +0::12::2::1424380312 +0::15::1::1424380312 +0::17::1::1424380312 +0::19::1::1424380312 +0::21::1::1424380312 +0::23::1::1424380312 +0::26::3::1424380312 +0::27::1::1424380312 +0::28::1::1424380312 +0::29::1::1424380312 +0::30::1::1424380312 +0::31::1::1424380312 +0::34::1::1424380312 +0::37::1::1424380312 +0::41::2::1424380312 +0::44::1::1424380312 +0::45::2::1424380312 +0::46::1::1424380312 +0::47::1::1424380312 +0::48::1::1424380312 +0::50::1::1424380312 +0::51::1::1424380312 +0::54::1::1424380312 +0::55::1::1424380312 +0::59::2::1424380312 +0::61::2::1424380312 +0::64::1::1424380312 +0::67::1::1424380312 +0::68::1::1424380312 +0::69::1::1424380312 +0::71::1::1424380312 +0::72::1::1424380312 +0::77::2::1424380312 +0::79::1::1424380312 +0::83::1::1424380312 +0::87::1::1424380312 +0::89::2::1424380312 +0::91::3::1424380312 +0::92::4::1424380312 +0::94::1::1424380312 +0::95::2::1424380312 +0::96::1::1424380312 +0::98::1::1424380312 +0::99::1::1424380312 +1::2::2::1424380312 +1::3::1::1424380312 +1::4::2::1424380312 +1::6::1::1424380312 +1::9::3::1424380312 +1::12::1::1424380312 +1::13::1::1424380312 +1::14::1::1424380312 +1::16::1::1424380312 +1::19::1::1424380312 +1::21::3::1424380312 +1::27::1::1424380312 +1::28::3::1424380312 +1::33::1::1424380312 +1::36::2::1424380312 +1::37::1::1424380312 +1::40::1::1424380312 +1::41::2::1424380312 +1::43::1::1424380312 +1::44::1::1424380312 +1::47::1::1424380312 +1::50::1::1424380312 +1::54::1::1424380312 +1::56::2::1424380312 +1::57::1::1424380312 +1::58::1::1424380312 +1::60::1::1424380312 +1::62::4::1424380312 +1::63::1::1424380312 +1::67::1::1424380312 +1::68::4::1424380312 +1::70::2::1424380312 +1::72::1::1424380312 +1::73::1::1424380312 +1::74::2::1424380312 +1::76::1::1424380312 +1::77::3::1424380312 +1::78::1::1424380312 +1::81::1::1424380312 +1::82::1::1424380312 +1::85::3::1424380312 +1::86::2::1424380312 +1::88::2::1424380312 +1::91::1::1424380312 +1::92::2::1424380312 +1::93::1::1424380312 +1::94::2::1424380312 +1::96::1::1424380312 +1::97::1::1424380312 +2::4::3::1424380312 +2::6::1::1424380312 +2::8::5::1424380312 +2::9::1::1424380312 +2::10::1::1424380312 +2::12::3::1424380312 +2::13::1::1424380312 +2::15::2::1424380312 +2::18::2::1424380312 +2::19::4::1424380312 +2::22::1::1424380312 +2::26::1::1424380312 +2::28::1::1424380312 +2::34::4::1424380312 +2::35::1::1424380312 +2::37::5::1424380312 +2::38::1::1424380312 +2::39::5::1424380312 +2::40::4::1424380312 +2::47::1::1424380312 +2::50::1::1424380312 +2::52::2::1424380312 +2::54::1::1424380312 +2::55::1::1424380312 +2::57::2::1424380312 +2::58::2::1424380312 +2::59::1::1424380312 +2::61::1::1424380312 +2::62::1::1424380312 +2::64::1::1424380312 +2::65::1::1424380312 +2::66::3::1424380312 +2::68::1::1424380312 +2::71::3::1424380312 +2::76::1::1424380312 +2::77::1::1424380312 +2::78::1::1424380312 +2::80::1::1424380312 +2::83::5::1424380312 +2::85::1::1424380312 +2::87::2::1424380312 +2::88::1::1424380312 +2::89::4::1424380312 +2::90::1::1424380312 +2::92::4::1424380312 +2::93::5::1424380312 +3::0::1::1424380312 +3::1::1::1424380312 +3::2::1::1424380312 +3::7::3::1424380312 +3::8::3::1424380312 +3::9::1::1424380312 +3::14::1::1424380312 +3::15::1::1424380312 +3::16::1::1424380312 +3::18::4::1424380312 +3::19::1::1424380312 +3::24::3::1424380312 +3::26::1::1424380312 +3::29::3::1424380312 +3::33::1::1424380312 +3::34::3::1424380312 +3::35::1::1424380312 +3::36::3::1424380312 +3::37::1::1424380312 +3::38::2::1424380312 +3::43::1::1424380312 +3::44::1::1424380312 +3::46::1::1424380312 +3::47::1::1424380312 +3::51::5::1424380312 +3::52::3::1424380312 +3::56::1::1424380312 +3::58::1::1424380312 +3::60::3::1424380312 +3::62::1::1424380312 +3::65::2::1424380312 +3::66::1::1424380312 +3::67::1::1424380312 +3::68::2::1424380312 +3::70::1::1424380312 +3::72::2::1424380312 +3::76::3::1424380312 +3::79::3::1424380312 +3::80::4::1424380312 +3::81::1::1424380312 +3::83::1::1424380312 +3::84::1::1424380312 +3::86::1::1424380312 +3::87::2::1424380312 +3::88::4::1424380312 +3::89::1::1424380312 +3::91::1::1424380312 +3::94::3::1424380312 +4::1::1::1424380312 +4::6::1::1424380312 +4::8::1::1424380312 +4::9::1::1424380312 +4::10::1::1424380312 +4::11::1::1424380312 +4::12::1::1424380312 +4::13::1::1424380312 +4::14::2::1424380312 +4::15::1::1424380312 +4::17::1::1424380312 +4::20::1::1424380312 +4::22::1::1424380312 +4::23::1::1424380312 +4::24::1::1424380312 +4::29::4::1424380312 +4::30::1::1424380312 +4::31::1::1424380312 +4::34::1::1424380312 +4::35::1::1424380312 +4::36::1::1424380312 +4::39::2::1424380312 +4::40::3::1424380312 +4::41::4::1424380312 +4::43::2::1424380312 +4::44::1::1424380312 +4::45::1::1424380312 +4::46::1::1424380312 +4::47::1::1424380312 +4::49::2::1424380312 +4::50::1::1424380312 +4::51::1::1424380312 +4::52::4::1424380312 +4::54::1::1424380312 +4::55::1::1424380312 +4::60::3::1424380312 +4::61::1::1424380312 +4::62::4::1424380312 +4::63::3::1424380312 +4::65::1::1424380312 +4::67::2::1424380312 +4::69::1::1424380312 +4::70::4::1424380312 +4::71::1::1424380312 +4::73::1::1424380312 +4::78::1::1424380312 +4::84::1::1424380312 +4::85::1::1424380312 +4::87::3::1424380312 +4::88::3::1424380312 +4::89::2::1424380312 +4::96::1::1424380312 +4::97::1::1424380312 +4::98::1::1424380312 +4::99::1::1424380312 +5::0::1::1424380312 +5::1::1::1424380312 +5::4::1::1424380312 +5::5::1::1424380312 +5::8::1::1424380312 +5::9::3::1424380312 +5::10::2::1424380312 +5::13::3::1424380312 +5::15::1::1424380312 +5::19::1::1424380312 +5::20::3::1424380312 +5::21::2::1424380312 +5::23::3::1424380312 +5::27::1::1424380312 +5::28::1::1424380312 +5::29::1::1424380312 +5::31::1::1424380312 +5::36::3::1424380312 +5::38::2::1424380312 +5::39::1::1424380312 +5::42::1::1424380312 +5::48::3::1424380312 +5::49::4::1424380312 +5::50::3::1424380312 +5::51::1::1424380312 +5::52::1::1424380312 +5::54::1::1424380312 +5::55::5::1424380312 +5::56::3::1424380312 +5::58::1::1424380312 +5::60::1::1424380312 +5::61::1::1424380312 +5::64::3::1424380312 +5::65::2::1424380312 +5::68::4::1424380312 +5::70::1::1424380312 +5::71::1::1424380312 +5::72::1::1424380312 +5::74::1::1424380312 +5::79::1::1424380312 +5::81::2::1424380312 +5::84::1::1424380312 +5::85::1::1424380312 +5::86::1::1424380312 +5::88::1::1424380312 +5::90::4::1424380312 +5::91::2::1424380312 +5::95::2::1424380312 +5::99::1::1424380312 +6::0::1::1424380312 +6::1::1::1424380312 +6::2::3::1424380312 +6::5::1::1424380312 +6::6::1::1424380312 +6::9::1::1424380312 +6::10::1::1424380312 +6::15::2::1424380312 +6::16::2::1424380312 +6::17::1::1424380312 +6::18::1::1424380312 +6::20::1::1424380312 +6::21::1::1424380312 +6::22::1::1424380312 +6::24::1::1424380312 +6::25::5::1424380312 +6::26::1::1424380312 +6::28::1::1424380312 +6::30::1::1424380312 +6::33::1::1424380312 +6::38::1::1424380312 +6::39::1::1424380312 +6::43::4::1424380312 +6::44::1::1424380312 +6::45::1::1424380312 +6::48::1::1424380312 +6::49::1::1424380312 +6::50::1::1424380312 +6::53::1::1424380312 +6::54::1::1424380312 +6::55::1::1424380312 +6::56::1::1424380312 +6::58::4::1424380312 +6::59::1::1424380312 +6::60::1::1424380312 +6::61::3::1424380312 +6::63::3::1424380312 +6::66::1::1424380312 +6::67::3::1424380312 +6::68::1::1424380312 +6::69::1::1424380312 +6::71::2::1424380312 +6::73::1::1424380312 +6::75::1::1424380312 +6::77::1::1424380312 +6::79::1::1424380312 +6::81::1::1424380312 +6::84::1::1424380312 +6::85::3::1424380312 +6::86::1::1424380312 +6::87::1::1424380312 +6::88::1::1424380312 +6::89::1::1424380312 +6::91::2::1424380312 +6::94::1::1424380312 +6::95::2::1424380312 +6::96::1::1424380312 +7::1::1::1424380312 +7::2::2::1424380312 +7::3::1::1424380312 +7::4::1::1424380312 +7::7::1::1424380312 +7::10::1::1424380312 +7::11::2::1424380312 +7::14::2::1424380312 +7::15::1::1424380312 +7::16::1::1424380312 +7::18::1::1424380312 +7::21::1::1424380312 +7::22::1::1424380312 +7::23::1::1424380312 +7::25::5::1424380312 +7::26::1::1424380312 +7::29::4::1424380312 +7::30::1::1424380312 +7::31::3::1424380312 +7::32::1::1424380312 +7::33::1::1424380312 +7::35::1::1424380312 +7::37::2::1424380312 +7::39::3::1424380312 +7::40::2::1424380312 +7::42::2::1424380312 +7::44::1::1424380312 +7::45::2::1424380312 +7::47::4::1424380312 +7::48::1::1424380312 +7::49::1::1424380312 +7::53::1::1424380312 +7::54::1::1424380312 +7::55::1::1424380312 +7::56::1::1424380312 +7::59::1::1424380312 +7::61::2::1424380312 +7::62::3::1424380312 +7::63::2::1424380312 +7::66::1::1424380312 +7::67::3::1424380312 +7::74::1::1424380312 +7::75::1::1424380312 +7::76::3::1424380312 +7::77::1::1424380312 +7::81::1::1424380312 +7::82::1::1424380312 +7::84::2::1424380312 +7::85::4::1424380312 +7::86::1::1424380312 +7::92::2::1424380312 +7::96::1::1424380312 +7::97::1::1424380312 +7::98::1::1424380312 +8::0::1::1424380312 +8::2::4::1424380312 +8::3::2::1424380312 +8::4::2::1424380312 +8::5::1::1424380312 +8::7::1::1424380312 +8::9::1::1424380312 +8::11::1::1424380312 +8::15::1::1424380312 +8::18::1::1424380312 +8::19::1::1424380312 +8::21::1::1424380312 +8::29::5::1424380312 +8::31::3::1424380312 +8::33::1::1424380312 +8::35::1::1424380312 +8::36::1::1424380312 +8::40::2::1424380312 +8::44::1::1424380312 +8::45::1::1424380312 +8::50::1::1424380312 +8::51::1::1424380312 +8::52::5::1424380312 +8::53::5::1424380312 +8::54::1::1424380312 +8::55::1::1424380312 +8::56::1::1424380312 +8::58::4::1424380312 +8::60::3::1424380312 +8::62::4::1424380312 +8::64::1::1424380312 +8::67::3::1424380312 +8::69::1::1424380312 +8::71::1::1424380312 +8::72::3::1424380312 +8::77::3::1424380312 +8::78::1::1424380312 +8::79::1::1424380312 +8::83::1::1424380312 +8::85::5::1424380312 +8::86::1::1424380312 +8::88::1::1424380312 +8::90::1::1424380312 +8::92::2::1424380312 +8::95::4::1424380312 +8::96::3::1424380312 +8::97::1::1424380312 +8::98::1::1424380312 +8::99::1::1424380312 +9::2::3::1424380312 +9::3::1::1424380312 +9::4::1::1424380312 +9::5::1::1424380312 +9::6::1::1424380312 +9::7::5::1424380312 +9::9::1::1424380312 +9::12::1::1424380312 +9::14::3::1424380312 +9::15::1::1424380312 +9::19::1::1424380312 +9::21::1::1424380312 +9::22::1::1424380312 +9::24::1::1424380312 +9::25::1::1424380312 +9::26::1::1424380312 +9::30::3::1424380312 +9::32::4::1424380312 +9::35::2::1424380312 +9::36::2::1424380312 +9::37::2::1424380312 +9::38::1::1424380312 +9::39::1::1424380312 +9::43::3::1424380312 +9::49::5::1424380312 +9::50::3::1424380312 +9::53::1::1424380312 +9::54::1::1424380312 +9::58::1::1424380312 +9::59::1::1424380312 +9::60::1::1424380312 +9::61::1::1424380312 +9::63::3::1424380312 +9::64::3::1424380312 +9::68::1::1424380312 +9::69::1::1424380312 +9::70::3::1424380312 +9::71::1::1424380312 +9::73::2::1424380312 +9::75::1::1424380312 +9::77::2::1424380312 +9::81::2::1424380312 +9::82::1::1424380312 +9::83::1::1424380312 +9::84::1::1424380312 +9::86::1::1424380312 +9::87::4::1424380312 +9::88::1::1424380312 +9::90::3::1424380312 +9::94::2::1424380312 +9::95::3::1424380312 +9::97::2::1424380312 +9::98::1::1424380312 +10::0::3::1424380312 +10::2::4::1424380312 +10::4::3::1424380312 +10::7::1::1424380312 +10::8::1::1424380312 +10::10::1::1424380312 +10::13::2::1424380312 +10::14::1::1424380312 +10::16::2::1424380312 +10::17::1::1424380312 +10::18::1::1424380312 +10::21::1::1424380312 +10::22::1::1424380312 +10::24::1::1424380312 +10::25::3::1424380312 +10::28::1::1424380312 +10::35::1::1424380312 +10::36::1::1424380312 +10::37::1::1424380312 +10::38::1::1424380312 +10::39::1::1424380312 +10::40::4::1424380312 +10::41::2::1424380312 +10::42::3::1424380312 +10::43::1::1424380312 +10::49::3::1424380312 +10::50::1::1424380312 +10::51::1::1424380312 +10::52::1::1424380312 +10::55::2::1424380312 +10::56::1::1424380312 +10::58::1::1424380312 +10::63::1::1424380312 +10::66::1::1424380312 +10::67::2::1424380312 +10::68::1::1424380312 +10::75::1::1424380312 +10::77::1::1424380312 +10::79::1::1424380312 +10::86::1::1424380312 +10::89::3::1424380312 +10::90::1::1424380312 +10::97::1::1424380312 +10::98::1::1424380312 +11::0::1::1424380312 +11::6::2::1424380312 +11::9::1::1424380312 +11::10::1::1424380312 +11::11::1::1424380312 +11::12::1::1424380312 +11::13::4::1424380312 +11::16::1::1424380312 +11::18::5::1424380312 +11::19::4::1424380312 +11::20::1::1424380312 +11::21::1::1424380312 +11::22::1::1424380312 +11::23::5::1424380312 +11::25::1::1424380312 +11::27::5::1424380312 +11::30::5::1424380312 +11::32::5::1424380312 +11::35::3::1424380312 +11::36::2::1424380312 +11::37::2::1424380312 +11::38::4::1424380312 +11::39::1::1424380312 +11::40::1::1424380312 +11::41::1::1424380312 +11::43::2::1424380312 +11::45::1::1424380312 +11::47::1::1424380312 +11::48::5::1424380312 +11::50::4::1424380312 +11::51::3::1424380312 +11::59::1::1424380312 +11::61::1::1424380312 +11::62::1::1424380312 +11::64::1::1424380312 +11::66::4::1424380312 +11::67::1::1424380312 +11::69::5::1424380312 +11::70::1::1424380312 +11::71::3::1424380312 +11::72::3::1424380312 +11::75::3::1424380312 +11::76::1::1424380312 +11::77::1::1424380312 +11::78::1::1424380312 +11::79::5::1424380312 +11::80::3::1424380312 +11::81::4::1424380312 +11::82::1::1424380312 +11::86::1::1424380312 +11::88::1::1424380312 +11::89::1::1424380312 +11::90::4::1424380312 +11::94::2::1424380312 +11::97::3::1424380312 +11::99::1::1424380312 +12::2::1::1424380312 +12::4::1::1424380312 +12::6::1::1424380312 +12::7::3::1424380312 +12::8::1::1424380312 +12::14::1::1424380312 +12::15::2::1424380312 +12::16::4::1424380312 +12::17::5::1424380312 +12::18::2::1424380312 +12::21::1::1424380312 +12::22::2::1424380312 +12::23::3::1424380312 +12::24::1::1424380312 +12::25::1::1424380312 +12::27::5::1424380312 +12::30::2::1424380312 +12::31::4::1424380312 +12::35::5::1424380312 +12::38::1::1424380312 +12::41::1::1424380312 +12::44::2::1424380312 +12::45::1::1424380312 +12::50::4::1424380312 +12::51::1::1424380312 +12::52::1::1424380312 +12::53::1::1424380312 +12::54::1::1424380312 +12::56::2::1424380312 +12::57::1::1424380312 +12::60::1::1424380312 +12::63::1::1424380312 +12::64::5::1424380312 +12::66::3::1424380312 +12::67::1::1424380312 +12::70::1::1424380312 +12::72::1::1424380312 +12::74::1::1424380312 +12::75::1::1424380312 +12::77::1::1424380312 +12::78::1::1424380312 +12::79::3::1424380312 +12::82::2::1424380312 +12::83::1::1424380312 +12::84::1::1424380312 +12::85::1::1424380312 +12::86::1::1424380312 +12::87::1::1424380312 +12::88::1::1424380312 +12::91::3::1424380312 +12::92::1::1424380312 +12::94::4::1424380312 +12::95::2::1424380312 +12::96::1::1424380312 +12::98::2::1424380312 +13::0::1::1424380312 +13::3::1::1424380312 +13::4::2::1424380312 +13::5::1::1424380312 +13::6::1::1424380312 +13::12::1::1424380312 +13::14::2::1424380312 +13::15::1::1424380312 +13::17::1::1424380312 +13::18::3::1424380312 +13::20::1::1424380312 +13::21::1::1424380312 +13::22::1::1424380312 +13::26::1::1424380312 +13::27::1::1424380312 +13::29::3::1424380312 +13::31::1::1424380312 +13::33::1::1424380312 +13::40::2::1424380312 +13::43::2::1424380312 +13::44::1::1424380312 +13::45::1::1424380312 +13::49::1::1424380312 +13::51::1::1424380312 +13::52::2::1424380312 +13::53::3::1424380312 +13::54::1::1424380312 +13::62::1::1424380312 +13::63::2::1424380312 +13::64::1::1424380312 +13::68::1::1424380312 +13::71::1::1424380312 +13::72::3::1424380312 +13::73::1::1424380312 +13::74::3::1424380312 +13::77::2::1424380312 +13::78::1::1424380312 +13::79::2::1424380312 +13::83::3::1424380312 +13::85::1::1424380312 +13::86::1::1424380312 +13::87::2::1424380312 +13::88::2::1424380312 +13::90::1::1424380312 +13::93::4::1424380312 +13::94::1::1424380312 +13::98::1::1424380312 +13::99::1::1424380312 +14::1::1::1424380312 +14::3::3::1424380312 +14::4::1::1424380312 +14::5::1::1424380312 +14::6::1::1424380312 +14::7::1::1424380312 +14::9::1::1424380312 +14::10::1::1424380312 +14::11::1::1424380312 +14::12::1::1424380312 +14::13::1::1424380312 +14::14::3::1424380312 +14::15::1::1424380312 +14::16::1::1424380312 +14::17::1::1424380312 +14::20::1::1424380312 +14::21::1::1424380312 +14::24::1::1424380312 +14::25::2::1424380312 +14::27::1::1424380312 +14::28::1::1424380312 +14::29::5::1424380312 +14::31::3::1424380312 +14::34::1::1424380312 +14::36::1::1424380312 +14::37::2::1424380312 +14::39::2::1424380312 +14::40::1::1424380312 +14::44::1::1424380312 +14::45::1::1424380312 +14::47::3::1424380312 +14::48::1::1424380312 +14::49::1::1424380312 +14::51::1::1424380312 +14::52::5::1424380312 +14::53::3::1424380312 +14::54::1::1424380312 +14::55::1::1424380312 +14::56::1::1424380312 +14::62::4::1424380312 +14::63::5::1424380312 +14::67::3::1424380312 +14::68::1::1424380312 +14::69::3::1424380312 +14::71::1::1424380312 +14::72::4::1424380312 +14::73::1::1424380312 +14::76::5::1424380312 +14::79::1::1424380312 +14::82::1::1424380312 +14::83::1::1424380312 +14::88::1::1424380312 +14::93::3::1424380312 +14::94::1::1424380312 +14::95::2::1424380312 +14::96::4::1424380312 +14::98::1::1424380312 +15::0::1::1424380312 +15::1::4::1424380312 +15::2::1::1424380312 +15::5::2::1424380312 +15::6::1::1424380312 +15::7::1::1424380312 +15::13::1::1424380312 +15::14::1::1424380312 +15::15::1::1424380312 +15::17::2::1424380312 +15::19::2::1424380312 +15::22::2::1424380312 +15::23::2::1424380312 +15::25::1::1424380312 +15::26::3::1424380312 +15::27::1::1424380312 +15::28::2::1424380312 +15::29::1::1424380312 +15::32::1::1424380312 +15::33::2::1424380312 +15::34::1::1424380312 +15::35::2::1424380312 +15::36::1::1424380312 +15::37::1::1424380312 +15::39::1::1424380312 +15::42::1::1424380312 +15::46::5::1424380312 +15::48::2::1424380312 +15::50::2::1424380312 +15::51::1::1424380312 +15::52::1::1424380312 +15::58::1::1424380312 +15::62::1::1424380312 +15::64::3::1424380312 +15::65::2::1424380312 +15::72::1::1424380312 +15::73::1::1424380312 +15::74::1::1424380312 +15::79::1::1424380312 +15::80::1::1424380312 +15::81::1::1424380312 +15::82::2::1424380312 +15::85::1::1424380312 +15::87::1::1424380312 +15::91::2::1424380312 +15::96::1::1424380312 +15::97::1::1424380312 +15::98::3::1424380312 +16::2::1::1424380312 +16::5::3::1424380312 +16::6::2::1424380312 +16::7::1::1424380312 +16::9::1::1424380312 +16::12::1::1424380312 +16::14::1::1424380312 +16::15::1::1424380312 +16::19::1::1424380312 +16::21::2::1424380312 +16::29::4::1424380312 +16::30::2::1424380312 +16::32::1::1424380312 +16::34::1::1424380312 +16::36::1::1424380312 +16::38::1::1424380312 +16::46::1::1424380312 +16::47::3::1424380312 +16::48::1::1424380312 +16::49::1::1424380312 +16::50::1::1424380312 +16::51::5::1424380312 +16::54::5::1424380312 +16::55::1::1424380312 +16::56::2::1424380312 +16::57::1::1424380312 +16::60::1::1424380312 +16::63::2::1424380312 +16::65::1::1424380312 +16::67::1::1424380312 +16::72::1::1424380312 +16::74::1::1424380312 +16::80::1::1424380312 +16::81::1::1424380312 +16::82::1::1424380312 +16::85::5::1424380312 +16::86::1::1424380312 +16::90::5::1424380312 +16::91::1::1424380312 +16::93::1::1424380312 +16::94::3::1424380312 +16::95::2::1424380312 +16::96::3::1424380312 +16::98::3::1424380312 +16::99::1::1424380312 +17::2::1::1424380312 +17::3::1::1424380312 +17::6::1::1424380312 +17::10::4::1424380312 +17::11::1::1424380312 +17::13::2::1424380312 +17::17::5::1424380312 +17::19::1::1424380312 +17::20::5::1424380312 +17::22::4::1424380312 +17::28::1::1424380312 +17::29::1::1424380312 +17::33::1::1424380312 +17::34::1::1424380312 +17::35::2::1424380312 +17::37::1::1424380312 +17::38::1::1424380312 +17::45::1::1424380312 +17::46::5::1424380312 +17::47::1::1424380312 +17::49::3::1424380312 +17::51::1::1424380312 +17::55::5::1424380312 +17::56::3::1424380312 +17::57::1::1424380312 +17::58::1::1424380312 +17::59::1::1424380312 +17::60::1::1424380312 +17::63::1::1424380312 +17::66::1::1424380312 +17::68::4::1424380312 +17::69::1::1424380312 +17::70::1::1424380312 +17::72::1::1424380312 +17::73::3::1424380312 +17::78::1::1424380312 +17::79::1::1424380312 +17::82::2::1424380312 +17::84::1::1424380312 +17::90::5::1424380312 +17::91::3::1424380312 +17::92::1::1424380312 +17::93::1::1424380312 +17::94::4::1424380312 +17::95::2::1424380312 +17::97::1::1424380312 +18::1::1::1424380312 +18::4::3::1424380312 +18::5::2::1424380312 +18::6::1::1424380312 +18::7::1::1424380312 +18::10::1::1424380312 +18::11::4::1424380312 +18::12::2::1424380312 +18::13::1::1424380312 +18::15::1::1424380312 +18::18::1::1424380312 +18::20::1::1424380312 +18::21::2::1424380312 +18::22::1::1424380312 +18::23::2::1424380312 +18::25::1::1424380312 +18::26::1::1424380312 +18::27::1::1424380312 +18::28::5::1424380312 +18::29::1::1424380312 +18::31::1::1424380312 +18::32::1::1424380312 +18::36::1::1424380312 +18::38::5::1424380312 +18::39::5::1424380312 +18::40::1::1424380312 +18::42::1::1424380312 +18::43::1::1424380312 +18::44::4::1424380312 +18::46::1::1424380312 +18::47::1::1424380312 +18::48::1::1424380312 +18::51::2::1424380312 +18::55::1::1424380312 +18::56::1::1424380312 +18::57::1::1424380312 +18::62::1::1424380312 +18::63::1::1424380312 +18::66::3::1424380312 +18::67::1::1424380312 +18::70::1::1424380312 +18::75::1::1424380312 +18::76::3::1424380312 +18::77::1::1424380312 +18::80::3::1424380312 +18::81::3::1424380312 +18::82::1::1424380312 +18::83::5::1424380312 +18::84::1::1424380312 +18::97::1::1424380312 +18::98::1::1424380312 +18::99::2::1424380312 +19::0::1::1424380312 +19::1::1::1424380312 +19::2::1::1424380312 +19::4::1::1424380312 +19::6::2::1424380312 +19::11::1::1424380312 +19::12::1::1424380312 +19::14::1::1424380312 +19::23::1::1424380312 +19::26::1::1424380312 +19::31::1::1424380312 +19::32::4::1424380312 +19::33::1::1424380312 +19::34::1::1424380312 +19::37::1::1424380312 +19::38::1::1424380312 +19::41::1::1424380312 +19::43::1::1424380312 +19::45::1::1424380312 +19::48::1::1424380312 +19::49::1::1424380312 +19::50::2::1424380312 +19::53::2::1424380312 +19::54::3::1424380312 +19::55::1::1424380312 +19::56::2::1424380312 +19::58::1::1424380312 +19::61::1::1424380312 +19::62::1::1424380312 +19::63::1::1424380312 +19::64::1::1424380312 +19::65::1::1424380312 +19::69::2::1424380312 +19::72::1::1424380312 +19::74::3::1424380312 +19::76::1::1424380312 +19::78::1::1424380312 +19::79::1::1424380312 +19::81::1::1424380312 +19::82::1::1424380312 +19::84::1::1424380312 +19::86::1::1424380312 +19::87::2::1424380312 +19::90::4::1424380312 +19::93::1::1424380312 +19::94::4::1424380312 +19::95::2::1424380312 +19::96::1::1424380312 +19::98::4::1424380312 +20::0::1::1424380312 +20::1::1::1424380312 +20::2::2::1424380312 +20::4::2::1424380312 +20::6::1::1424380312 +20::8::1::1424380312 +20::12::1::1424380312 +20::21::2::1424380312 +20::22::5::1424380312 +20::24::2::1424380312 +20::25::1::1424380312 +20::26::1::1424380312 +20::29::2::1424380312 +20::30::2::1424380312 +20::32::2::1424380312 +20::39::1::1424380312 +20::40::1::1424380312 +20::41::2::1424380312 +20::45::2::1424380312 +20::48::1::1424380312 +20::50::1::1424380312 +20::51::3::1424380312 +20::53::3::1424380312 +20::55::1::1424380312 +20::57::2::1424380312 +20::60::1::1424380312 +20::61::1::1424380312 +20::64::1::1424380312 +20::66::1::1424380312 +20::70::2::1424380312 +20::72::1::1424380312 +20::73::2::1424380312 +20::75::4::1424380312 +20::76::1::1424380312 +20::77::4::1424380312 +20::78::1::1424380312 +20::79::1::1424380312 +20::84::2::1424380312 +20::85::2::1424380312 +20::88::3::1424380312 +20::89::1::1424380312 +20::90::3::1424380312 +20::91::1::1424380312 +20::92::2::1424380312 +20::93::1::1424380312 +20::94::4::1424380312 +20::97::1::1424380312 +21::0::1::1424380312 +21::2::4::1424380312 +21::3::1::1424380312 +21::7::2::1424380312 +21::11::1::1424380312 +21::12::1::1424380312 +21::13::1::1424380312 +21::14::3::1424380312 +21::17::1::1424380312 +21::19::1::1424380312 +21::20::1::1424380312 +21::21::1::1424380312 +21::22::1::1424380312 +21::23::1::1424380312 +21::24::1::1424380312 +21::27::1::1424380312 +21::29::5::1424380312 +21::30::2::1424380312 +21::38::1::1424380312 +21::40::2::1424380312 +21::43::3::1424380312 +21::44::1::1424380312 +21::45::1::1424380312 +21::46::1::1424380312 +21::48::1::1424380312 +21::51::1::1424380312 +21::53::5::1424380312 +21::54::1::1424380312 +21::55::1::1424380312 +21::56::1::1424380312 +21::58::3::1424380312 +21::59::3::1424380312 +21::64::1::1424380312 +21::66::1::1424380312 +21::68::1::1424380312 +21::71::1::1424380312 +21::73::1::1424380312 +21::74::4::1424380312 +21::80::1::1424380312 +21::81::1::1424380312 +21::83::1::1424380312 +21::84::1::1424380312 +21::85::3::1424380312 +21::87::4::1424380312 +21::89::2::1424380312 +21::92::2::1424380312 +21::96::3::1424380312 +21::99::1::1424380312 +22::0::1::1424380312 +22::3::2::1424380312 +22::5::2::1424380312 +22::6::2::1424380312 +22::9::1::1424380312 +22::10::1::1424380312 +22::11::1::1424380312 +22::13::1::1424380312 +22::14::1::1424380312 +22::16::1::1424380312 +22::18::3::1424380312 +22::19::1::1424380312 +22::22::5::1424380312 +22::25::1::1424380312 +22::26::1::1424380312 +22::29::3::1424380312 +22::30::5::1424380312 +22::32::4::1424380312 +22::33::1::1424380312 +22::35::1::1424380312 +22::36::3::1424380312 +22::37::1::1424380312 +22::40::1::1424380312 +22::41::3::1424380312 +22::44::1::1424380312 +22::45::2::1424380312 +22::48::1::1424380312 +22::51::5::1424380312 +22::55::1::1424380312 +22::56::2::1424380312 +22::60::3::1424380312 +22::61::1::1424380312 +22::62::4::1424380312 +22::63::1::1424380312 +22::65::1::1424380312 +22::66::1::1424380312 +22::68::4::1424380312 +22::69::4::1424380312 +22::70::3::1424380312 +22::71::1::1424380312 +22::74::5::1424380312 +22::75::5::1424380312 +22::78::1::1424380312 +22::80::3::1424380312 +22::81::1::1424380312 +22::82::1::1424380312 +22::84::1::1424380312 +22::86::1::1424380312 +22::87::3::1424380312 +22::88::5::1424380312 +22::90::2::1424380312 +22::92::3::1424380312 +22::95::2::1424380312 +22::96::2::1424380312 +22::98::4::1424380312 +22::99::1::1424380312 +23::0::1::1424380312 +23::2::1::1424380312 +23::4::1::1424380312 +23::6::2::1424380312 +23::10::4::1424380312 +23::12::1::1424380312 +23::13::4::1424380312 +23::14::1::1424380312 +23::15::1::1424380312 +23::18::4::1424380312 +23::22::2::1424380312 +23::23::4::1424380312 +23::24::1::1424380312 +23::25::1::1424380312 +23::26::1::1424380312 +23::27::5::1424380312 +23::28::1::1424380312 +23::29::1::1424380312 +23::30::4::1424380312 +23::32::5::1424380312 +23::33::2::1424380312 +23::36::3::1424380312 +23::37::1::1424380312 +23::38::1::1424380312 +23::39::1::1424380312 +23::43::1::1424380312 +23::48::5::1424380312 +23::49::5::1424380312 +23::50::4::1424380312 +23::53::1::1424380312 +23::55::5::1424380312 +23::57::1::1424380312 +23::59::1::1424380312 +23::60::1::1424380312 +23::61::1::1424380312 +23::64::4::1424380312 +23::65::5::1424380312 +23::66::2::1424380312 +23::67::1::1424380312 +23::68::3::1424380312 +23::69::1::1424380312 +23::72::1::1424380312 +23::73::3::1424380312 +23::77::1::1424380312 +23::82::2::1424380312 +23::83::1::1424380312 +23::84::1::1424380312 +23::85::1::1424380312 +23::87::3::1424380312 +23::88::1::1424380312 +23::95::2::1424380312 +23::97::1::1424380312 +24::4::1::1424380312 +24::6::3::1424380312 +24::7::1::1424380312 +24::10::2::1424380312 +24::12::1::1424380312 +24::15::1::1424380312 +24::19::1::1424380312 +24::24::1::1424380312 +24::27::3::1424380312 +24::30::5::1424380312 +24::31::1::1424380312 +24::32::3::1424380312 +24::33::1::1424380312 +24::37::1::1424380312 +24::39::1::1424380312 +24::40::1::1424380312 +24::42::1::1424380312 +24::43::3::1424380312 +24::45::2::1424380312 +24::46::1::1424380312 +24::47::1::1424380312 +24::48::1::1424380312 +24::49::1::1424380312 +24::50::1::1424380312 +24::52::5::1424380312 +24::57::1::1424380312 +24::59::4::1424380312 +24::63::4::1424380312 +24::65::1::1424380312 +24::66::1::1424380312 +24::67::1::1424380312 +24::68::3::1424380312 +24::69::5::1424380312 +24::71::1::1424380312 +24::72::4::1424380312 +24::77::4::1424380312 +24::78::1::1424380312 +24::80::1::1424380312 +24::82::1::1424380312 +24::84::1::1424380312 +24::86::1::1424380312 +24::87::1::1424380312 +24::88::2::1424380312 +24::89::1::1424380312 +24::90::5::1424380312 +24::91::1::1424380312 +24::92::1::1424380312 +24::94::2::1424380312 +24::95::1::1424380312 +24::96::5::1424380312 +24::98::1::1424380312 +24::99::1::1424380312 +25::1::3::1424380312 +25::2::1::1424380312 +25::7::1::1424380312 +25::9::1::1424380312 +25::12::3::1424380312 +25::16::3::1424380312 +25::17::1::1424380312 +25::18::1::1424380312 +25::20::1::1424380312 +25::22::1::1424380312 +25::23::1::1424380312 +25::26::2::1424380312 +25::29::1::1424380312 +25::30::1::1424380312 +25::31::2::1424380312 +25::33::4::1424380312 +25::34::3::1424380312 +25::35::2::1424380312 +25::36::1::1424380312 +25::37::1::1424380312 +25::40::1::1424380312 +25::41::1::1424380312 +25::43::1::1424380312 +25::47::4::1424380312 +25::50::1::1424380312 +25::51::1::1424380312 +25::53::1::1424380312 +25::56::1::1424380312 +25::58::2::1424380312 +25::64::2::1424380312 +25::67::2::1424380312 +25::68::1::1424380312 +25::70::1::1424380312 +25::71::4::1424380312 +25::73::1::1424380312 +25::74::1::1424380312 +25::76::1::1424380312 +25::79::1::1424380312 +25::82::1::1424380312 +25::84::2::1424380312 +25::85::1::1424380312 +25::91::3::1424380312 +25::92::1::1424380312 +25::94::1::1424380312 +25::95::1::1424380312 +25::97::2::1424380312 +26::0::1::1424380312 +26::1::1::1424380312 +26::2::1::1424380312 +26::3::1::1424380312 +26::4::4::1424380312 +26::5::2::1424380312 +26::6::3::1424380312 +26::7::5::1424380312 +26::13::3::1424380312 +26::14::1::1424380312 +26::16::1::1424380312 +26::18::3::1424380312 +26::20::1::1424380312 +26::21::3::1424380312 +26::22::5::1424380312 +26::23::5::1424380312 +26::24::5::1424380312 +26::27::1::1424380312 +26::31::1::1424380312 +26::35::1::1424380312 +26::36::4::1424380312 +26::40::1::1424380312 +26::44::1::1424380312 +26::45::2::1424380312 +26::47::1::1424380312 +26::48::1::1424380312 +26::49::3::1424380312 +26::50::2::1424380312 +26::52::1::1424380312 +26::54::4::1424380312 +26::55::1::1424380312 +26::57::3::1424380312 +26::58::1::1424380312 +26::61::1::1424380312 +26::62::2::1424380312 +26::66::1::1424380312 +26::68::4::1424380312 +26::71::1::1424380312 +26::73::4::1424380312 +26::76::1::1424380312 +26::81::3::1424380312 +26::85::1::1424380312 +26::86::3::1424380312 +26::88::5::1424380312 +26::91::1::1424380312 +26::94::5::1424380312 +26::95::1::1424380312 +26::96::1::1424380312 +26::97::1::1424380312 +27::0::1::1424380312 +27::9::1::1424380312 +27::10::1::1424380312 +27::18::4::1424380312 +27::19::3::1424380312 +27::20::1::1424380312 +27::22::2::1424380312 +27::24::2::1424380312 +27::25::1::1424380312 +27::27::3::1424380312 +27::28::1::1424380312 +27::29::1::1424380312 +27::31::1::1424380312 +27::33::3::1424380312 +27::40::1::1424380312 +27::42::1::1424380312 +27::43::1::1424380312 +27::44::3::1424380312 +27::45::1::1424380312 +27::51::3::1424380312 +27::52::1::1424380312 +27::55::3::1424380312 +27::57::1::1424380312 +27::59::1::1424380312 +27::60::1::1424380312 +27::61::1::1424380312 +27::64::1::1424380312 +27::66::3::1424380312 +27::68::1::1424380312 +27::70::1::1424380312 +27::71::2::1424380312 +27::72::1::1424380312 +27::75::3::1424380312 +27::78::1::1424380312 +27::80::3::1424380312 +27::82::1::1424380312 +27::83::3::1424380312 +27::86::1::1424380312 +27::87::2::1424380312 +27::90::1::1424380312 +27::91::1::1424380312 +27::92::1::1424380312 +27::93::1::1424380312 +27::94::2::1424380312 +27::95::1::1424380312 +27::98::1::1424380312 +28::0::3::1424380312 +28::1::1::1424380312 +28::2::4::1424380312 +28::3::1::1424380312 +28::6::1::1424380312 +28::7::1::1424380312 +28::12::5::1424380312 +28::13::2::1424380312 +28::14::1::1424380312 +28::15::1::1424380312 +28::17::1::1424380312 +28::19::3::1424380312 +28::20::1::1424380312 +28::23::3::1424380312 +28::24::3::1424380312 +28::27::1::1424380312 +28::29::1::1424380312 +28::33::1::1424380312 +28::34::1::1424380312 +28::36::1::1424380312 +28::38::2::1424380312 +28::39::2::1424380312 +28::44::1::1424380312 +28::45::1::1424380312 +28::49::4::1424380312 +28::50::1::1424380312 +28::52::1::1424380312 +28::54::1::1424380312 +28::56::1::1424380312 +28::57::3::1424380312 +28::58::1::1424380312 +28::59::1::1424380312 +28::60::1::1424380312 +28::62::3::1424380312 +28::63::1::1424380312 +28::65::1::1424380312 +28::75::1::1424380312 +28::78::1::1424380312 +28::81::5::1424380312 +28::82::4::1424380312 +28::83::1::1424380312 +28::85::1::1424380312 +28::88::2::1424380312 +28::89::4::1424380312 +28::90::1::1424380312 +28::92::5::1424380312 +28::94::1::1424380312 +28::95::2::1424380312 +28::98::1::1424380312 +28::99::1::1424380312 +29::3::1::1424380312 +29::4::1::1424380312 +29::5::1::1424380312 +29::7::2::1424380312 +29::9::1::1424380312 +29::10::3::1424380312 +29::11::1::1424380312 +29::13::3::1424380312 +29::14::1::1424380312 +29::15::1::1424380312 +29::17::3::1424380312 +29::19::3::1424380312 +29::22::3::1424380312 +29::23::4::1424380312 +29::25::1::1424380312 +29::29::1::1424380312 +29::31::1::1424380312 +29::32::4::1424380312 +29::33::2::1424380312 +29::36::2::1424380312 +29::38::3::1424380312 +29::39::1::1424380312 +29::42::1::1424380312 +29::46::5::1424380312 +29::49::3::1424380312 +29::51::2::1424380312 +29::59::1::1424380312 +29::61::1::1424380312 +29::62::1::1424380312 +29::67::1::1424380312 +29::68::3::1424380312 +29::69::1::1424380312 +29::70::1::1424380312 +29::74::1::1424380312 +29::75::1::1424380312 +29::79::2::1424380312 +29::80::1::1424380312 +29::81::2::1424380312 +29::83::1::1424380312 +29::85::1::1424380312 +29::86::1::1424380312 +29::90::4::1424380312 +29::93::1::1424380312 +29::94::4::1424380312 +29::97::1::1424380312 +29::99::1::1424380312 diff --git a/data/mllib/gmm_data.txt b/data/mllib/gmm_data.txt new file mode 100644 index 0000000000000..934ee4a83a2df --- /dev/null +++ b/data/mllib/gmm_data.txt @@ -0,0 +1,2000 @@ + 2.59470454e+00 2.12298217e+00 + 1.15807024e+00 -1.46498723e-01 + 2.46206638e+00 6.19556894e-01 + -5.54845070e-01 -7.24700066e-01 + -3.23111426e+00 -1.42579084e+00 + 3.02978115e+00 7.87121753e-01 + 1.97365907e+00 1.15914704e+00 + -6.44852101e+00 -3.18154314e+00 + 1.30963349e+00 1.62866434e-01 + 4.26482541e+00 2.15547996e+00 + 3.79927257e+00 1.50572445e+00 + 4.17452609e-01 -6.74032760e-01 + 4.21117627e-01 4.45590255e-01 + -2.80425571e+00 -7.77150554e-01 + 2.55928797e+00 7.03954218e-01 + 1.32554059e+00 -9.46663152e-01 + -3.39691439e+00 -1.49005743e+00 + -2.26542270e-01 3.60052515e-02 + 1.04994198e+00 5.29825685e-01 + -1.51566882e+00 -1.86264432e-01 + -3.27928172e-01 -7.60859110e-01 + -3.18054866e-01 3.97719805e-01 + 1.65579418e-01 -3.47232033e-01 + 6.47162333e-01 4.96059961e-02 + -2.80776647e-01 4.79418757e-01 + 7.45069752e-01 1.20790281e-01 + 2.13604102e-01 1.59542555e-01 + -3.08860224e+00 -1.43259870e+00 + 8.97066497e-01 1.10206801e+00 + -2.23918874e-01 -1.07267267e+00 + 2.51525708e+00 2.84761973e-01 + 9.98052532e-01 1.08333783e+00 + 1.76705588e+00 8.18866778e-01 + 5.31555163e-02 -1.90111151e-01 + -2.17405059e+00 7.21854582e-02 + -2.13772505e+00 -3.62010387e-01 + 2.95974057e+00 1.31602381e+00 + 2.74053561e+00 1.61781757e+00 + 6.68135448e-01 2.86586009e-01 + 2.82323739e+00 1.74437257e+00 + 8.11540288e-01 5.50744478e-01 + 4.10050897e-01 5.10668402e-03 + 9.58626136e-01 -3.49633680e-01 + 4.66599798e+00 1.49964894e+00 + 4.94507794e-01 2.58928077e-01 + -2.36029742e+00 -1.61042909e+00 + -4.99306804e-01 -8.04984769e-01 + 1.07448510e+00 9.39605828e-01 + -1.80448949e+00 -1.05983264e+00 + -3.22353821e-01 1.73612093e-01 + 1.85418702e+00 1.15640643e+00 + 6.93794163e-01 6.59993560e-01 + 1.99399102e+00 1.44547123e+00 + 3.38866124e+00 1.23379290e+00 + -4.24067720e+00 -1.22264282e+00 + 6.03230201e-02 2.95232729e-01 + -3.59341813e+00 -7.17453726e-01 + 4.87447372e-01 -2.00733911e-01 + 1.20149195e+00 4.07880197e-01 + -2.13331464e+00 -4.58518077e-01 + -3.84091083e+00 -1.71553950e+00 + -5.37279250e-01 2.64822629e-02 + -2.10155227e+00 -1.32558103e+00 + -1.71318897e+00 -7.12098563e-01 + -1.46280695e+00 -1.84868337e-01 + -3.59785325e+00 -1.54832434e+00 + -5.77528081e-01 -5.78580857e-01 + 3.14734283e-01 5.80184639e-01 + -2.71164714e+00 -1.19379432e+00 + 1.09634489e+00 7.20143887e-01 + -3.05527722e+00 -1.47774064e+00 + 6.71753586e-01 7.61350020e-01 + 3.98294144e+00 1.54166484e+00 + -3.37220384e+00 -2.21332064e+00 + 1.81222914e+00 7.41212752e-01 + 2.71458282e-01 1.36329078e-01 + -3.97815359e-01 1.16766886e-01 + -1.70192814e+00 -9.75851571e-01 + -3.46803804e+00 -1.09965988e+00 + -1.69649627e+00 -5.76045801e-01 + -1.02485636e-01 -8.81841246e-01 + -3.24194667e-02 2.55429276e-01 + -2.75343168e+00 -1.51366320e+00 + -2.78676702e+00 -5.22360489e-01 + 1.70483164e+00 1.19769805e+00 + 4.92022579e-01 3.24944706e-01 + 2.48768464e+00 1.00055363e+00 + 4.48786400e-01 7.63902870e-01 + 2.93862696e+00 1.73809968e+00 + -3.55019305e+00 -1.97875558e+00 + 1.74270784e+00 6.90229224e-01 + 5.13391994e-01 4.58374016e-01 + 1.78379499e+00 9.08026381e-01 + 1.75814147e+00 7.41449784e-01 + -2.30687792e-01 3.91009729e-01 + 3.92271353e+00 1.44006290e+00 + 2.93361679e-01 -4.99886375e-03 + 2.47902690e-01 -7.49542503e-01 + -3.97675355e-01 1.36824887e-01 + 3.56535953e+00 1.15181329e+00 + 3.22425301e+00 1.28702383e+00 + -2.94192478e-01 -2.42382557e-01 + 8.02068864e-01 -1.51671475e-01 + 8.54133530e-01 -4.89514885e-02 + -1.64316316e-01 -5.34642346e-01 + -6.08485405e-01 -2.10332352e-01 + -2.18940059e+00 -1.07024952e+00 + -1.71586960e+00 -2.83333492e-02 + 1.70200448e-01 -3.28031178e-01 + -1.97210346e+00 -5.39948532e-01 + 2.19500160e+00 1.05697170e+00 + -1.76239935e+00 -1.09377438e+00 + 1.68314744e+00 6.86491164e-01 + -2.99852288e+00 -1.46619067e+00 + -2.23769560e+00 -9.15008355e-01 + 9.46887516e-01 5.58410503e-01 + 5.02153123e-01 1.63851235e-01 + -9.70297062e-01 3.14625374e-01 + -1.29405593e+00 -8.20994131e-01 + 2.72516079e+00 7.85839947e-01 + 1.45788024e+00 3.37487353e-01 + -4.36292749e-01 -5.42150480e-01 + 2.21304711e+00 1.25254042e+00 + -1.20810271e-01 4.79632898e-01 + -3.30884511e+00 -1.50607586e+00 + -6.55882455e+00 -1.94231256e+00 + -3.17033630e+00 -9.94678930e-01 + 1.42043617e+00 7.28808957e-01 + -1.57546099e+00 -1.10320497e+00 + -3.22748754e+00 -1.64174579e+00 + 2.96776017e-03 -3.16191512e-02 + -2.25986054e+00 -6.13123197e-01 + 2.49434243e+00 7.73069183e-01 + 9.08494049e-01 -1.53926853e-01 + -2.80559090e+00 -1.37474221e+00 + 4.75224286e-01 2.53153674e-01 + 4.37644006e+00 8.49116998e-01 + 2.27282959e+00 6.16568202e-01 + 1.16006880e+00 1.65832798e-01 + -1.67163193e+00 -1.22555386e+00 + -1.38231118e+00 -7.29575504e-01 + -3.49922750e+00 -2.26446675e+00 + -3.73780110e-01 -1.90657869e-01 + 1.68627679e+00 1.05662987e+00 + -3.28891792e+00 -1.11080334e+00 + -2.59815798e+00 -1.51410198e+00 + -2.61203309e+00 -6.00143552e-01 + 6.58964943e-01 4.47216094e-01 + -2.26711381e+00 -7.26512923e-01 + -5.31429009e-02 -1.97925341e-02 + 3.19749807e+00 9.20425476e-01 + -1.37595787e+00 -6.58062732e-01 + 8.09900278e-01 -3.84286160e-01 + -5.07741280e+00 -1.97683808e+00 + -2.99764250e+00 -1.50753777e+00 + -9.87671815e-01 -4.63255889e-01 + 1.65390765e+00 6.73806615e-02 + 5.51252659e+00 2.69842267e+00 + -2.23724309e+00 -4.77624004e-01 + 4.99726228e+00 1.74690949e+00 + 1.75859162e-01 -1.49350995e-01 + 4.13382789e+00 1.31735161e+00 + 2.69058117e+00 4.87656923e-01 + 1.07180318e+00 1.01426954e+00 + 3.37216869e+00 1.05955377e+00 + -2.95006781e+00 -1.57048303e+00 + -2.46401648e+00 -8.37056374e-01 + 1.19012962e-01 7.54702770e-01 + 3.34142539e+00 4.81938295e-01 + 2.92643913e+00 1.04301050e+00 + 2.89697751e+00 1.37551442e+00 + -1.03094242e+00 2.20903962e-01 + -5.13914589e+00 -2.23355387e+00 + -8.81680780e-01 1.83590000e-01 + 2.82334775e+00 1.26650464e+00 + -2.81042540e-01 -3.26370240e-01 + 2.97995487e+00 8.34569452e-01 + -1.39857135e+00 -1.15798385e+00 + 4.27186506e+00 9.04253702e-01 + 6.98684517e-01 7.91167305e-01 + 3.52233095e+00 1.29976473e+00 + 2.21448029e+00 2.73213379e-01 + -3.13505683e-01 -1.20593774e-01 + 3.70571571e+00 1.06220876e+00 + 9.83881041e-01 5.67713803e-01 + -2.17897705e+00 2.52925205e-01 + 1.38734039e+00 4.61287066e-01 + -1.41181602e+00 -1.67248955e-02 + -1.69974639e+00 -7.17812071e-01 + -2.01005793e-01 -7.49662056e-01 + 1.69016336e+00 3.24687979e-01 + -2.03250179e+00 -2.76108460e-01 + 3.68776848e-01 4.12536941e-01 + 7.66238259e-01 -1.84750637e-01 + -2.73989147e-01 -1.72817250e-01 + -2.18623745e+00 -2.10906798e-01 + -1.39795625e-01 3.26066094e-02 + -2.73826912e-01 -6.67586097e-02 + -1.57880654e+00 -4.99395900e-01 + 4.55950908e+00 2.29410489e+00 + -7.36479631e-01 -1.57861857e-01 + 1.92082888e+00 1.05843391e+00 + 4.29192810e+00 1.38127810e+00 + 1.61852879e+00 1.95871986e-01 + -1.95027403e+00 -5.22448168e-01 + -1.67446281e+00 -9.41497162e-01 + 6.07097859e-01 3.44178029e-01 + -3.44004683e+00 -1.49258461e+00 + 2.72114752e+00 6.00728991e-01 + 8.80685522e-01 -2.53243336e-01 + 1.39254928e+00 3.42988512e-01 + 1.14194836e-01 -8.57945694e-02 + -1.49387332e+00 -7.60860481e-01 + -1.98053285e+00 -4.86039865e-01 + 3.56008568e+00 1.08438692e+00 + 2.27833961e-01 1.09441881e+00 + -1.16716710e+00 -6.54778242e-01 + 2.02156613e+00 5.42075758e-01 + 1.08429178e+00 -7.67420693e-01 + 6.63058455e-01 4.61680991e-01 + -1.06201537e+00 1.38862846e-01 + 3.08701875e+00 8.32580273e-01 + -4.96558108e-01 -2.47031257e-01 + 7.95109987e-01 7.59314147e-02 + -3.39903524e-01 8.71565566e-03 + 8.68351357e-01 4.78358641e-01 + 1.48750819e+00 7.63257420e-01 + -4.51224101e-01 -4.44056898e-01 + -3.02734750e-01 -2.98487961e-01 + 5.46846609e-01 7.02377629e-01 + 1.65129778e+00 3.74008231e-01 + -7.43336512e-01 3.95723531e-01 + -5.88446605e-01 -6.47520211e-01 + 3.58613167e+00 1.95024937e+00 + 3.11718883e+00 8.37984715e-01 + 1.80919244e+00 9.62644986e-01 + 5.43856371e-02 -5.86297543e-01 + -1.95186766e+00 -1.02624212e-01 + 8.95628057e-01 5.91812281e-01 + 4.97691627e-02 5.31137156e-01 + -1.07633113e+00 -2.47392788e-01 + -1.17257986e+00 -8.68528265e-01 + -8.19227665e-02 5.80579434e-03 + -2.86409787e-01 1.95812924e-01 + 1.10582671e+00 7.42853240e-01 + 4.06429774e+00 1.06557476e+00 + -3.42521792e+00 -7.74327139e-01 + 1.28468671e+00 6.20431661e-01 + 6.01201008e-01 -1.16799728e-01 + -1.85058727e-01 -3.76235293e-01 + 5.44083324e+00 2.98490868e+00 + 2.69273070e+00 7.83901153e-01 + 1.88938036e-01 -4.83222152e-01 + 1.05667256e+00 -2.57003165e-01 + 2.99711662e-01 -4.33131912e-01 + 7.73689216e-02 -1.78738364e-01 + 9.58326279e-01 6.38325706e-01 + -3.97727049e-01 2.27314759e-01 + 3.36098175e+00 1.12165237e+00 + 1.77804871e+00 6.46961933e-01 + -2.86945546e+00 -1.00395518e+00 + 3.03494815e+00 7.51814612e-01 + -1.43658194e+00 -3.55432244e-01 + -3.08455105e+00 -1.51535106e+00 + -1.55841975e+00 3.93454820e-02 + 7.96073412e-01 -3.11036969e-01 + -9.84125401e-01 -1.02064649e+00 + -7.75688143e+00 -3.65219926e+00 + 1.53816429e+00 7.65926670e-01 + -4.92712738e-01 2.32244240e-02 + -1.93166919e+00 -1.07701304e+00 + 2.03029875e-02 -7.54055699e-01 + 2.52177489e+00 1.01544979e+00 + 3.65109048e-01 -9.48328494e-01 + -1.28849143e-01 2.51947174e-01 + -1.02428075e+00 -9.37767116e-01 + -3.04179748e+00 -9.97926994e-01 + -2.51986980e+00 -1.69117413e+00 + -1.24900838e+00 -4.16179917e-01 + 2.77943992e+00 1.22842327e+00 + -4.37434557e+00 -1.70182693e+00 + -1.60019319e+00 -4.18345639e-01 + -1.67613646e+00 -9.44087262e-01 + -9.00843245e-01 8.26378089e-02 + 3.29770621e-01 -9.07870444e-01 + -2.84650535e+00 -9.00155396e-01 + 1.57111705e+00 7.07432268e-01 + 1.24948552e+00 1.04812849e-01 + 1.81440558e+00 9.53545082e-01 + -1.74915794e+00 -1.04606288e+00 + 1.20593269e+00 -1.12607147e-02 + 1.36004919e-01 -1.09828044e+00 + 2.57480693e-01 3.34941541e-01 + 7.78775385e-01 -5.32494732e-01 + -1.79155126e+00 -6.29994129e-01 + -1.75706839e+00 -8.35100126e-01 + 4.29512012e-01 7.81426910e-02 + 3.08349370e-01 -1.27359861e-01 + 1.05560329e+00 4.55150640e-01 + 1.95662574e+00 1.17593217e+00 + 8.77376632e-01 6.57866662e-01 + 7.71311255e-01 9.15134334e-02 + -6.36978275e+00 -2.55874241e+00 + -2.98335339e+00 -1.59567024e+00 + -3.67104587e-01 1.85315291e-01 + 1.95347407e+00 -7.15503113e-02 + 8.45556363e-01 6.51256415e-02 + 9.42868521e-01 3.56647624e-01 + 2.99321875e+00 1.07505254e+00 + -2.91030538e-01 -3.77637183e-01 + 1.62870918e+00 3.37563671e-01 + 2.05773173e-01 3.43337416e-01 + -8.40879199e-01 -1.35600767e-01 + 1.38101624e+00 5.99253495e-01 + -6.93715607e+00 -2.63580662e+00 + -1.04423404e+00 -8.32865050e-01 + 1.33448476e+00 1.04863475e+00 + 6.01675207e-01 1.98585194e-01 + 2.31233993e+00 7.98628331e-01 + 1.85201313e-01 -1.76070247e+00 + 1.92006354e+00 8.45737582e-01 + 1.06320415e+00 2.93426068e-01 + -1.20360141e+00 -1.00301288e+00 + 1.95926629e+00 6.26643532e-01 + 6.04483978e-02 5.72643059e-01 + -1.04568563e+00 -5.91021496e-01 + 2.62300678e+00 9.50997831e-01 + -4.04610275e-01 3.73150879e-01 + 2.26371902e+00 8.73627529e-01 + 2.12545313e+00 7.90640352e-01 + 7.72181917e-03 1.65718952e-02 + 1.00422340e-01 -2.05562936e-01 + -1.22989802e+00 -1.01841681e-01 + 3.09064082e+00 1.04288010e+00 + 5.18274167e+00 1.34749259e+00 + -8.32075153e-01 -1.97592029e-01 + 3.84126764e-02 5.58171345e-01 + 4.99560727e-01 -4.26154438e-02 + 4.79071151e+00 2.19728942e+00 + -2.78437968e+00 -1.17812590e+00 + -2.22804226e+00 -4.31174255e-01 + 8.50762292e-01 -1.06445261e-01 + 1.10812830e+00 -2.59118812e-01 + -2.91450155e-01 6.42802679e-01 + -1.38631532e-01 -5.88585623e-01 + -5.04120983e-01 -2.17094915e-01 + 3.41410820e+00 1.67897767e+00 + -2.23697326e+00 -6.62735244e-01 + -3.55961064e-01 -1.27647226e-01 + -3.55568274e+00 -2.49011369e+00 + -8.77586408e-01 -9.38268065e-03 + 1.52382384e-01 -5.62155760e-01 + 1.55885574e-01 1.07617069e-01 + -8.37129973e-01 -5.22259081e-01 + -2.92741750e+00 -1.35049428e+00 + -3.54670781e-01 5.69205952e-02 + 2.21030255e+00 1.34689986e+00 + 1.60787722e+00 5.75984706e-01 + 1.32294221e+00 5.31577509e-01 + 7.05672928e-01 3.34241244e-01 + 1.41406179e+00 1.15783408e+00 + -6.92172228e-01 -2.84817896e-01 + 3.28358655e-01 -2.66910083e-01 + 1.68013644e-01 -4.28016549e-02 + 2.07365974e+00 7.76496211e-01 + -3.92974907e-01 2.46796730e-01 + -5.76078636e-01 3.25676963e-01 + -1.82547204e-01 -5.06410543e-01 + 3.04754906e+00 1.16174496e+00 + -3.01090632e+00 -1.09195183e+00 + -1.44659696e+00 -6.87838682e-01 + 2.11395861e+00 9.10495785e-01 + 1.40962871e+00 1.13568678e+00 + -1.66653234e-01 -2.10012503e-01 + 3.17456029e+00 9.74502922e-01 + 2.15944820e+00 8.62807189e-01 + -3.45418719e+00 -1.33647548e+00 + -3.41357732e+00 -8.47048920e-01 + -3.06702448e-01 -6.64280634e-01 + -2.86930714e-01 -1.35268264e-01 + -3.15835557e+00 -5.43439253e-01 + 2.49541440e-01 -4.71733570e-01 + 2.71933912e+00 4.13308399e-01 + -2.43787038e+00 -1.08050547e+00 + -4.90234490e-01 -6.64069865e-01 + 8.99524451e-02 5.76180541e-01 + 5.00500404e+00 2.12125521e+00 + -1.73107940e-01 -2.28506575e-02 + 5.44938858e-01 -1.29523352e-01 + 5.13526842e+00 1.68785993e+00 + 1.70228304e+00 1.02601138e+00 + 3.58957507e+00 1.54396196e+00 + 1.85615738e+00 4.92916197e-01 + 2.55772147e+00 7.88438908e-01 + -1.57008279e+00 -4.17377300e-01 + -1.42548604e+00 -3.63684860e-01 + -8.52026118e-01 2.72052686e-01 + -5.10563077e+00 -2.35665994e+00 + -2.95517031e+00 -1.84945297e+00 + -2.91947959e+00 -1.66016784e+00 + -4.21462387e+00 -1.41131535e+00 + 6.59901121e-01 4.87156314e-01 + -9.75352532e-01 -4.50231285e-01 + -5.94084444e-01 -1.16922670e+00 + 7.50554615e-01 -9.83692552e-01 + 1.07054926e+00 2.77143030e-01 + -3.88079578e-01 -4.17737309e-02 + -9.59373733e-01 -8.85454886e-01 + -7.53560665e-02 -5.16223870e-02 + 9.84108158e-01 -5.89290700e-02 + 1.87272961e-01 -4.34238391e-01 + 6.86509981e-01 -3.15116460e-01 + -1.07762538e+00 6.58984161e-02 + 6.09266592e-01 6.91808473e-02 + -8.30529954e-01 -7.00454791e-01 + -9.13179464e-01 -6.31712891e-01 + 7.68744851e-01 1.09840676e+00 + -1.07606690e+00 -8.78390282e-01 + -1.71038184e+00 -5.73606033e-01 + 8.75982765e-01 3.66343143e-01 + -7.04919009e-01 -8.49182590e-01 + -1.00274668e+00 -7.99573611e-01 + -1.05562848e+00 -5.84060076e-01 + 4.03490015e+00 1.28679206e+00 + -3.53484804e+00 -1.71381255e+00 + 2.31527363e-01 1.04179397e-01 + -3.58592392e-02 3.74895739e-01 + 3.92253428e+00 1.81852726e+00 + -7.27384249e-01 -6.45605128e-01 + 4.65678097e+00 2.41379899e+00 + 1.16750534e+00 7.60718205e-01 + 1.15677059e+00 7.96225550e-01 + -1.42920261e+00 -4.66946295e-01 + 3.71148192e+00 1.88060191e+00 + 2.44052407e+00 3.84472199e-01 + -1.64535035e+00 -8.94530036e-01 + -3.69608753e+00 -1.36402754e+00 + 2.24419208e+00 9.69744889e-01 + 2.54822427e+00 1.22613039e+00 + 3.77484909e-01 -5.98521878e-01 + -3.61521175e+00 -1.11123912e+00 + 3.28113127e+00 1.52551775e+00 + -3.51030902e+00 -1.53913980e+00 + -2.44874505e+00 -6.30246005e-01 + -3.42516153e-01 -5.07352665e-01 + 1.09110502e+00 6.36821628e-01 + -2.49434967e+00 -8.02827146e-01 + 1.41763139e+00 -3.46591820e-01 + 1.61108619e+00 5.93871102e-01 + 3.97371717e+00 1.35552499e+00 + -1.33437177e+00 -2.83908670e-01 + -1.41606483e+00 -1.76402601e-01 + 2.23945322e-01 -1.77157065e-01 + 2.60271569e+00 2.40778251e-01 + -2.82213895e-02 1.98255474e-01 + 4.20727940e+00 1.31490863e+00 + 3.36944889e+00 1.57566635e+00 + 3.53049396e+00 1.73579350e+00 + -1.29170202e+00 -1.64196290e+00 + 9.27295604e-01 9.98808036e-01 + 1.75321843e-01 -2.83267817e-01 + -2.19069578e+00 -1.12814358e+00 + 1.66606031e+00 7.68006933e-01 + -7.13826035e-01 5.20881684e-02 + -3.43821888e+00 -2.36137021e+00 + -5.93210310e-01 1.21843813e-01 + -4.09800822e+00 -1.39893953e+00 + 2.74110954e+00 1.52728606e+00 + 1.72652512e+00 -1.25435113e-01 + 1.97722357e+00 6.40667481e-01 + 4.18635780e-01 3.57018509e-01 + -1.78303569e+00 -2.11864764e-01 + -3.52809366e+00 -2.58794450e-01 + -4.72407090e+00 -1.63870734e+00 + 1.73917807e+00 8.73251829e-01 + 4.37979356e-01 8.49210569e-01 + 3.93791881e+00 1.76269490e+00 + 2.79065411e+00 1.04019042e+00 + -8.47426142e-01 -3.40136892e-01 + -4.24389181e+00 -1.80253120e+00 + -1.86675870e+00 -7.64558265e-01 + 9.46212675e-01 -7.77681445e-02 + -2.82448462e+00 -1.33592449e+00 + -2.57938567e+00 -1.56554690e+00 + -2.71615767e+00 -6.27667233e-01 + -1.55999166e+00 -5.81013466e-01 + -4.24696864e-01 -7.44673250e-01 + 1.67592970e+00 7.68164292e-01 + 8.48455216e-01 -6.05681126e-01 + 6.12575454e+00 1.65607584e+00 + 1.38207327e+00 2.39261863e-01 + 3.13364450e+00 1.17154698e+00 + 1.71694858e+00 1.26744905e+00 + -1.61746367e+00 -8.80098073e-01 + -8.52196756e-01 -9.27299728e-01 + -1.51562462e-01 -8.36552490e-02 + -7.04792753e-01 -1.24726713e-02 + -3.35265757e+00 -1.82176312e+00 + 3.32173170e-01 -1.33405580e-01 + 4.95841013e-01 4.58292712e-01 + 1.57713955e+00 7.79272991e-01 + 2.09743109e+00 9.23542557e-01 + 3.90450311e-03 -8.42873164e-01 + 2.59519038e+00 7.56479591e-01 + -5.77643976e-01 -2.36401904e-01 + -5.22310654e-01 1.34187830e-01 + -2.22096086e+00 -7.75507719e-01 + 1.35907831e+00 7.80197510e-01 + 3.80355868e+00 1.16983476e+00 + 3.82746596e+00 1.31417718e+00 + 3.30451183e+00 1.55398159e+00 + -3.42917814e-01 -8.62281222e-02 + -2.59093020e+00 -9.29883526e-01 + 1.40928562e+00 1.08398346e+00 + 1.54400137e-01 3.35881092e-01 + 1.59171586e+00 1.18855802e+00 + -5.25164002e-01 -1.03104220e-01 + 2.20067959e+00 1.37074713e+00 + 6.97860830e-01 6.27718548e-01 + -4.59743507e-01 1.36061163e-01 + -1.04691963e-01 -2.16271727e-01 + -1.08905573e+00 -5.95510769e-01 + -1.00826983e+00 -5.38509162e-02 + -3.16402719e+00 -1.33414216e+00 + 1.47870874e-01 1.75234619e-01 + -2.57078234e-01 7.03316889e-02 + 1.81073945e+00 4.26901462e-01 + 2.65476530e+00 6.74217273e-01 + 1.27539811e+00 6.22914081e-01 + -3.76750499e-01 -1.20629449e+00 + 1.00177595e+00 -1.40660091e-01 + -2.98919265e+00 -1.65145013e+00 + -2.21557682e+00 -8.11123452e-01 + -3.22635378e+00 -1.65639056e+00 + -2.72868553e+00 -1.02812087e+00 + 1.26042797e+00 8.49005248e-01 + -9.38318534e-01 -9.87588651e-01 + 3.38013194e-01 -1.00237461e-01 + 1.91175691e+00 8.48716369e-01 + 4.30244344e-01 6.05539915e-02 + 2.21783435e+00 3.03268204e-01 + 1.78019576e+00 1.27377108e+00 + 1.59733274e+00 4.40674687e-02 + 3.97428484e+00 2.20881566e+00 + -2.41108677e+00 -6.01410418e-01 + -2.50796499e+00 -5.71169866e-01 + -3.71957427e+00 -1.38195726e+00 + -1.57992670e+00 1.32068593e-01 + -1.35278851e+00 -6.39349270e-01 + 1.23075932e+00 2.40445409e-01 + 1.35606530e+00 4.33180078e-01 + 9.60968518e-02 2.26734255e-01 + 6.22975063e-01 5.03431915e-02 + -1.47624851e+00 -3.60568238e-01 + -2.49337808e+00 -1.15083052e+00 + 2.15717792e+00 1.03071559e+00 + -3.07814376e-02 1.38700314e-02 + 4.52049499e-02 -4.86409775e-01 + 2.58231061e+00 1.14327809e-01 + 1.10999138e+00 -5.18568405e-01 + -2.19426443e-01 -5.37505538e-01 + -4.44740298e-01 6.78099955e-01 + 4.03379080e+00 1.49825720e+00 + -5.13182408e-01 -4.90201950e-01 + -6.90139716e-01 1.63875126e-01 + -8.17281461e-01 2.32155064e-01 + -2.92357619e-01 -8.02573544e-01 + -1.80769841e+00 -7.58907326e-01 + 2.16981590e+00 1.06728873e+00 + 1.98995203e-01 -6.84176682e-02 + -2.39546753e+00 -2.92873789e-01 + -4.24251021e+00 -1.46255564e+00 + -5.01411291e-01 -5.95712813e-03 + 2.68085809e+00 1.42883780e+00 + -4.13289873e+00 -1.62729388e+00 + 1.87957843e+00 3.63341638e-01 + -1.15270744e+00 -3.03563774e-01 + -4.43994248e+00 -2.97323905e+00 + -7.17067733e-01 -7.08349542e-01 + -3.28870393e+00 -1.19263863e+00 + -7.55325944e-01 -5.12703329e-01 + -2.07291938e+00 -2.65025085e-01 + -7.50073814e-01 -1.70771041e-01 + -8.77381404e-01 -5.47417325e-01 + -5.33725862e-01 5.15837119e-01 + 8.45056431e-01 2.82125560e-01 + -1.59598637e+00 -1.38743235e+00 + 1.41362902e+00 1.06407789e+00 + 1.02584504e+00 -3.68219466e-01 + -1.04644488e+00 -1.48769392e-01 + 2.66990191e+00 8.57633492e-01 + -1.84251857e+00 -9.82430175e-01 + 9.71404204e-01 -2.81934209e-01 + -2.50177989e+00 -9.21260335e-01 + -1.31060074e+00 -5.84488113e-01 + -2.12129400e-01 -3.06244708e-02 + -5.28933882e+00 -2.50663129e+00 + 1.90220541e+00 1.08662918e+00 + -3.99366086e-02 -6.87178973e-01 + -4.93417342e-01 4.37354182e-01 + 2.13494486e+00 1.37679569e+00 + 2.18396765e+00 5.81023868e-01 + -3.07866587e+00 -1.45384974e+00 + 6.10894119e-01 -4.17050124e-01 + -1.88766952e+00 -8.86160058e-01 + 3.34527253e+00 1.78571260e+00 + 6.87769059e-01 -5.01157336e-01 + 2.60470837e+00 1.45853560e+00 + -6.49315691e-01 -9.16112805e-01 + -1.29817687e+00 -2.15924339e-01 + -1.20100409e-03 -4.03137422e-01 + -1.36471594e+00 -6.93266356e-01 + 1.38682062e+00 7.15131598e-01 + 2.47830103e+00 1.24862305e+00 + -2.78288147e+00 -1.03329235e+00 + -7.33443403e-01 -6.11041652e-01 + -4.12745671e-01 -5.96133390e-02 + -2.58632336e+00 -4.51557058e-01 + -1.16570367e+00 -1.27065510e+00 + 2.76187104e+00 2.21895451e-01 + -3.80443767e+00 -1.66319902e+00 + 9.84658633e-01 6.81475569e-01 + 9.33814584e-01 -4.89335563e-02 + -4.63427997e-01 1.72989539e-01 + 1.82401546e+00 3.60164021e-01 + -5.36521077e-01 -8.08691351e-01 + -1.37367030e+00 -1.02126160e+00 + -3.70310682e+00 -1.19840844e+00 + -1.51894242e+00 -3.89510223e-01 + -3.67347940e-01 -3.25540516e-02 + -1.00988595e+00 1.82802194e-01 + 2.01622795e+00 7.86367901e-01 + 1.02440231e+00 8.79780360e-01 + -3.05971480e+00 -8.40901527e-01 + 2.73909457e+00 1.20558628e+00 + 2.39559056e+00 1.10786694e+00 + 1.65471544e+00 7.33824651e-01 + 2.18546787e+00 6.41168955e-01 + 1.47152266e+00 3.91839132e-01 + 1.45811155e+00 5.21820495e-01 + -4.27531469e-02 -3.52343068e-03 + -9.54948010e-01 -1.52313876e-01 + 7.57151215e-01 -5.68728854e-03 + -8.46205751e-01 -7.54580229e-01 + 4.14493548e+00 1.45532780e+00 + 4.58688968e-01 -4.54012803e-02 + -1.49295381e+00 -4.57471758e-01 + 1.80020351e+00 8.13724973e-01 + -5.82727738e+00 -2.18269581e+00 + -2.09017809e+00 -1.18305177e+00 + -2.31628303e+00 -7.21600235e-01 + -8.09679091e-01 -1.49101752e-01 + 8.88005605e-01 8.57940857e-01 + -1.44148219e+00 -3.10926299e-01 + 3.68828186e-01 -3.08848059e-01 + -6.63267389e-01 -8.58950139e-02 + -1.14702569e+00 -6.32147854e-01 + -1.51741715e+00 -8.53330564e-01 + -1.33903718e+00 -1.45875547e-01 + 4.12485387e+00 1.85620435e+00 + -2.42353639e+00 -2.92669850e-01 + 1.88708583e+00 9.35984730e-01 + 2.15585179e+00 6.30469051e-01 + -1.13627973e-01 -1.62554045e-01 + 2.04540494e+00 1.36599834e+00 + 2.81591381e+00 1.60897941e+00 + 3.02736260e-02 3.83255815e-03 + 7.97634013e-02 -2.82035099e-01 + -3.24607473e-01 -5.30065956e-01 + -3.91862894e+00 -1.94083334e+00 + 1.56360901e+00 7.93882743e-01 + -1.03905772e+00 6.25590229e-01 + 2.54746492e+00 1.64233560e+00 + -4.80774423e-01 -8.92298032e-02 + 9.06979990e-02 1.05020427e+00 + -2.47521290e+00 -1.78275982e-01 + -3.91871729e-01 3.80285423e-01 + 1.00658382e+00 4.58947483e-01 + 4.68102941e-01 1.02992741e+00 + 4.44242568e-01 2.89870239e-01 + 3.29684452e+00 1.44677474e+00 + -2.24983007e+00 -9.65574499e-01 + -3.54453926e-01 -3.99020325e-01 + -3.87429665e+00 -1.90079739e+00 + 2.02656674e+00 1.12444894e+00 + 3.77011621e+00 1.43200852e+00 + 1.61259275e+00 4.65417399e-01 + 2.28725434e+00 6.79181395e-01 + 2.75421009e+00 2.27327345e+00 + -2.40894409e+00 -1.03926359e+00 + 1.52996651e-01 -2.73373046e-02 + -2.63218977e+00 -7.22802821e-01 + 2.77688169e+00 1.15310186e+00 + 1.18832341e+00 4.73457165e-01 + -2.35536326e+00 -1.08034554e+00 + -5.84221627e-01 1.03505984e-02 + 2.96730300e+00 1.33478306e+00 + -8.61947692e-01 6.09137051e-02 + 8.22343921e-01 -8.14155286e-02 + 1.75809015e+00 1.07921470e+00 + 1.19501279e+00 1.05309972e+00 + -1.75901792e+00 9.75320161e-02 + 1.64398635e+00 9.54384323e-01 + -2.21878052e-01 -3.64847144e-01 + -2.03128968e+00 -8.57866419e-01 + 1.86750633e+00 7.08524487e-01 + 8.03972976e-01 3.47404314e-01 + 3.41203749e+00 1.39810900e+00 + 4.22397681e-01 -6.41440488e-01 + -4.88493360e+00 -1.58967816e+00 + -1.67649284e-01 -1.08485915e-01 + 2.11489023e+00 1.50506158e+00 + -1.81639929e+00 -3.85542192e-01 + 2.24044819e-01 -1.45100577e-01 + -3.39262411e+00 -1.44394324e+00 + 1.68706599e+00 2.29199618e-01 + -1.94093257e+00 -1.65975814e-01 + 8.28143367e-01 5.92109281e-01 + -8.29587998e-01 -9.57130831e-01 + -1.50011401e+00 -8.36802092e-01 + 2.40770449e+00 9.32820177e-01 + 7.41391309e-02 3.12878473e-01 + 1.87745264e-01 6.19231425e-01 + 9.57622692e-01 -2.20640033e-01 + 3.18479243e+00 1.02986233e+00 + 2.43133846e+00 8.41302677e-01 + -7.09963834e-01 1.99718943e-01 + -2.88253498e-01 -3.62772094e-01 + 5.14052574e+00 1.79304595e+00 + -3.27930993e+00 -1.29177973e+00 + -1.16723536e+00 1.29519656e-01 + 1.04801056e+00 3.41508300e-01 + -3.99256195e+00 -2.51176471e+00 + -7.62824318e-01 -6.84242153e-01 + 2.71524986e-02 5.35157164e-02 + 3.26430102e+00 1.34887262e+00 + -1.72357766e+00 -4.94524388e-01 + -3.81149536e+00 -1.28121944e+00 + 3.36919354e+00 1.10672075e+00 + -3.14841757e+00 -7.10713767e-01 + -3.16463676e+00 -7.58558435e-01 + -2.44745969e+00 -1.08816514e+00 + 2.79173264e-01 -2.19652051e-02 + 4.15309883e-01 6.07502790e-01 + -9.51007417e-01 -5.83976336e-01 + -1.47929839e+00 -8.39850409e-01 + 2.38335703e+00 6.16055149e-01 + -7.47749031e-01 -5.56164928e-01 + -3.65643622e-01 -5.06684411e-01 + -1.76634163e+00 -7.86382097e-01 + 6.76372222e-01 -3.06592181e-01 + -1.33505058e+00 -1.18301441e-01 + 3.59660179e+00 2.00424178e+00 + -7.88912762e-02 8.71956146e-02 + 1.22656397e+00 1.18149583e+00 + 4.24919729e+00 1.20082355e+00 + 2.94607456e+00 1.00676505e+00 + 7.46061275e-02 4.41761753e-02 + -2.47738025e-02 1.92737701e-01 + -2.20509316e-01 -3.79163193e-01 + -3.50222190e-01 3.58727299e-01 + -3.64788014e+00 -1.36107312e+00 + 3.56062799e+00 9.27032742e-01 + 1.04317289e+00 6.08035970e-01 + 4.06718718e-01 3.00628051e-01 + 4.33158086e+00 2.25860714e+00 + 2.13917145e-01 -1.72757967e-01 + -1.40637998e+00 -1.14119465e+00 + 3.61554872e+00 1.87797348e+00 + 1.01726871e+00 5.70255097e-01 + -7.04902551e-01 2.16444147e-01 + -2.51492186e+00 -8.52997369e-01 + 1.85097530e+00 1.15124496e+00 + -8.67569714e-01 -3.05682432e-01 + 8.07550858e-01 5.88901608e-01 + 1.85186755e-01 -1.94589367e-01 + -1.23378238e+00 -7.84128347e-01 + -1.22713161e+00 -4.21218235e-01 + 2.97751165e-01 2.81055275e-01 + 4.77703554e+00 1.66265524e+00 + 2.51549669e+00 7.49980674e-01 + 2.76510822e-01 1.40456909e-01 + 1.98740905e+00 -1.79608212e-01 + 9.35429145e-01 8.44344180e-01 + -1.20854492e+00 -5.00598453e-01 + 2.29936219e+00 8.10236668e-01 + 6.92555544e-01 -2.65891331e-01 + -1.58050994e+00 2.31237821e-01 + -1.50864880e+00 -9.49661690e-01 + -1.27689206e+00 -7.18260016e-01 + -3.12517127e+00 -1.75587113e+00 + 8.16062912e-02 -6.56551804e-01 + -5.02479939e-01 -4.67162543e-01 + -5.47435788e+00 -2.47799576e+00 + 1.95872901e-02 5.80874076e-01 + -1.59064958e+00 -6.34554756e-01 + -3.77521478e+00 -1.74301790e+00 + 5.89628224e-01 8.55736553e-01 + -1.81903543e+00 -7.50011008e-01 + 1.38557775e+00 3.71490991e-01 + 9.70032652e-01 -7.11356016e-01 + 2.63539625e-01 -4.20994771e-01 + 2.12154222e+00 8.19081400e-01 + -6.56977937e-01 -1.37810098e-01 + 8.91309581e-01 2.77864361e-01 + -7.43693195e-01 -1.46293770e-01 + 2.24447769e+00 4.00911438e-01 + -2.25169262e-01 2.04148801e-02 + 1.68744684e+00 9.47573007e-01 + 2.73086373e-01 3.30877195e-01 + 5.54294414e+00 2.14198009e+00 + -8.49238733e-01 3.65603298e-02 + 2.39685712e+00 1.17951039e+00 + -2.58230528e+00 -5.52116673e-01 + 2.79785277e+00 2.88833717e-01 + -1.96576188e-01 1.11652123e+00 + -4.69383301e-01 1.96496282e-01 + -1.95011845e+00 -6.15235169e-01 + 1.03379890e-02 2.33701239e-01 + 4.18933607e-01 2.77939814e-01 + -1.18473337e+00 -4.10051126e-01 + -7.61499744e-01 -1.43658094e+00 + -1.65586092e+00 -3.41615303e-01 + -5.58523700e-02 -5.21837080e-01 + -2.40331088e+00 -2.64521583e-01 + 2.24925206e+00 6.79843335e-02 + 1.46360479e+00 1.04271443e+00 + -3.09255443e+00 -1.82548953e+00 + 2.11325841e+00 1.14996627e+00 + -8.70657797e-01 1.02461839e-01 + -5.71056521e-01 9.71232588e-02 + -3.37870752e+00 -1.54091877e+00 + 1.03907189e+00 -1.35661392e-01 + 8.40057486e-01 6.12172413e-02 + -1.30998234e+00 -1.34077226e+00 + 7.53744974e-01 1.49447350e-01 + 9.13995056e-01 -1.81227962e-01 + 2.28386229e-01 3.74498520e-01 + 2.54829151e-01 -2.88802704e-01 + 1.61709009e+00 2.09319193e-01 + -1.12579380e+00 -5.95955338e-01 + -2.69610726e+00 -2.76222736e-01 + -2.63773329e+00 -7.84491970e-01 + -2.62167427e+00 -1.54792874e+00 + -4.80639856e-01 -1.30582102e-01 + -1.26130891e+00 -8.86841840e-01 + -1.24951950e+00 -1.18182622e+00 + -1.40107574e+00 -9.13695575e-01 + 4.99872179e-01 4.69014702e-01 + -2.03550193e-02 -1.48859738e-01 + -1.50189069e+00 -2.97714278e-02 + -2.07846113e+00 -7.29937809e-01 + -5.50576792e-01 -7.03151525e-01 + -3.88069238e+00 -1.63215295e+00 + 2.97032988e+00 6.43571144e-01 + -1.85999273e-01 1.18107620e+00 + 1.79249709e+00 6.65356160e-01 + 2.68842472e+00 1.35703255e+00 + 1.07675417e+00 1.39845588e-01 + 8.01226349e-01 2.11392275e-01 + 9.64329379e-01 3.96146195e-01 + -8.22529511e-01 1.96080831e-01 + 1.92481841e+00 4.62985744e-01 + 3.69756927e-01 3.77135799e-01 + 1.19807835e+00 8.87715050e-01 + -1.01363587e+00 -2.48151636e-01 + 8.53071010e-01 4.96887868e-01 + -3.41120553e+00 -1.35401843e+00 + -2.64787381e+00 -1.08690563e+00 + -1.11416759e+00 -4.43848915e-01 + 1.46242648e+00 6.17106076e-02 + -7.52968881e-01 -9.20972209e-01 + -1.22492228e+00 -5.40327617e-01 + 1.08001827e+00 5.29593785e-01 + -2.58706464e-01 1.13022085e-01 + -4.27394011e-01 1.17864354e-02 + -3.20728413e+00 -1.71224737e-01 + 1.71398530e+00 8.68885893e-01 + 2.12067866e+00 1.45092772e+00 + 4.32782616e-01 -3.34117769e-01 + 7.80084374e-01 -1.35100217e-01 + -2.05547729e+00 -4.70217750e-01 + 2.38379736e+00 1.09186058e+00 + -2.80825477e+00 -1.03320187e+00 + 2.63434576e+00 1.15671733e+00 + -1.60936214e+00 1.91843035e-01 + -5.02298769e+00 -2.32820708e+00 + 1.90349195e+00 1.45215416e+00 + 3.00232888e-01 3.24412586e-01 + -2.46503943e+00 -1.19550010e+00 + 1.06304233e+00 2.20136246e-01 + -2.99101388e+00 -1.58299318e+00 + 2.30071719e+00 1.12881362e+00 + -2.37587247e+00 -8.08298336e-01 + 7.27006308e-01 3.80828984e-01 + 2.61199061e+00 1.56473491e+00 + 8.33936357e-01 -1.42189425e-01 + 3.13291605e+00 1.77771210e+00 + 2.21917371e+00 5.68427075e-01 + 2.38867649e+00 9.06637262e-01 + -6.92959466e+00 -3.57682881e+00 + 2.57904824e+00 5.93959108e-01 + 2.71452670e+00 1.34436199e+00 + 4.39988761e+00 2.13124672e+00 + 5.71783077e-01 5.08346173e-01 + -3.65399429e+00 -1.18192861e+00 + 4.46176453e-01 3.75685594e-02 + -2.97501495e+00 -1.69459236e+00 + 1.60855728e+00 9.20930014e-01 + -1.44270290e+00 -1.93922306e-01 + 1.67624229e+00 1.66233866e+00 + -1.42579598e+00 -1.44990145e-01 + 1.19923176e+00 4.58490278e-01 + -9.00068460e-01 5.09701825e-02 + -1.69391694e+00 -7.60070300e-01 + -1.36576440e+00 -5.24244256e-01 + -1.03016748e+00 -3.44625878e-01 + 2.40519313e+00 1.09947587e+00 + 1.50365433e+00 1.06464802e+00 + -1.07609727e+00 -3.68897187e-01 + 2.44969069e+00 1.28486192e+00 + -1.25610307e+00 -1.14644789e+00 + 2.05962899e+00 4.31162369e-01 + -7.15886908e-01 -6.11587804e-02 + -6.92354119e-01 -7.85019920e-01 + -1.63016508e+00 -5.96944975e-01 + 1.90352536e+00 1.28197457e+00 + -4.01535243e+00 -1.81934488e+00 + -1.07534435e+00 -2.10544784e-01 + 3.25500866e-01 7.69603661e-01 + 2.18443365e+00 6.59773335e-01 + 8.80856790e-01 6.39505913e-01 + -2.23956372e-01 -4.65940132e-01 + -1.06766519e+00 -5.38388505e-03 + 7.25556863e-01 -2.91123488e-01 + -4.69451411e-01 7.89182650e-02 + 2.58146587e+00 1.29653243e+00 + 1.53747468e-01 7.69239075e-01 + -4.61152262e-01 -4.04151413e-01 + 1.48183517e+00 8.10079506e-01 + -1.83402614e+00 -1.36939322e+00 + 1.49315501e+00 7.95225425e-01 + 1.41922346e+00 1.05582774e-01 + 1.57473493e-01 9.70795657e-01 + -2.67603254e+00 -7.48562280e-01 + -8.49156216e-01 -6.05762529e-03 + 1.12944274e+00 3.67741591e-01 + 1.94228071e-01 5.28188141e-01 + -3.65610158e-01 4.05851838e-01 + -1.98839111e+00 -1.38452764e+00 + 2.73765752e+00 8.24150530e-01 + 7.63728641e-01 3.51617707e-01 + 5.78307267e+00 1.68103612e+00 + 2.27547227e+00 3.60876164e-01 + -3.50681697e+00 -1.74429984e+00 + 4.01241184e+00 1.26227829e+00 + 2.44946343e+00 9.06119057e-01 + -2.96638941e+00 -9.01532322e-01 + 1.11267643e+00 -3.43333381e-01 + -6.61868994e-01 -3.44666391e-01 + -8.34917179e-01 5.69478372e-01 + -1.91888454e+00 -3.03791075e-01 + 1.50397636e+00 8.31961240e-01 + 6.12260198e+00 2.16851807e+00 + 1.34093127e+00 8.86649385e-01 + 1.48748519e+00 8.26273697e-01 + 7.62243068e-01 2.64841396e-01 + -2.17604986e+00 -3.54219958e-01 + 2.64708640e-01 -4.38136718e-02 + 1.44725372e+00 1.18499914e-01 + -6.71259446e-01 -1.19526851e-01 + 2.40134595e-01 -8.90042323e-02 + -3.57238199e+00 -1.23166201e+00 + -3.77626645e+00 -1.19533443e+00 + -3.81101035e-01 -4.94160532e-01 + -3.02758757e+00 -1.18436066e+00 + 2.59116298e-01 1.38023047e+00 + 4.17900116e+00 1.12065959e+00 + 1.54598848e+00 2.89806755e-01 + 1.00656475e+00 1.76974511e-01 + -4.15730234e-01 -6.22681694e-01 + -6.00903565e-01 -1.43256959e-01 + -6.03652508e-01 -5.09936379e-01 + -1.94096658e+00 -9.48789544e-01 + -1.74464105e+00 -8.50491590e-01 + 1.17652544e+00 1.88118317e+00 + 2.35507776e+00 1.44000205e+00 + 2.63067924e+00 1.06692988e+00 + 2.88805386e+00 1.23924715e+00 + 8.27595008e-01 5.75364692e-01 + 3.91384216e-01 9.72781920e-02 + -1.03866816e+00 -1.37567768e+00 + -1.34777969e+00 -8.40266025e-02 + -4.12904508e+00 -1.67618340e+00 + 1.27918111e+00 3.52085961e-01 + 4.15361174e-01 6.28896189e-01 + -7.00539496e-01 4.80447955e-02 + -1.62332639e+00 -5.98236485e-01 + 1.45957300e+00 1.00305154e+00 + -3.06875603e+00 -1.25897545e+00 + -1.94708176e+00 4.85143006e-01 + 3.55744156e+00 -1.07468822e+00 + 1.21602223e+00 1.28768827e-01 + 1.89093098e+00 -4.70835659e-01 + -6.55759125e+00 2.70114082e+00 + 8.96843535e-01 -3.98115252e-01 + 4.13450429e+00 -2.32069236e+00 + 2.37764218e+00 -1.09098890e+00 + -1.11388901e+00 6.27083097e-01 + -6.34116929e-01 4.62816387e-01 + 2.90203079e+00 -1.33589143e+00 + 3.17457598e+00 -5.13575945e-01 + -1.76362299e+00 5.71820693e-01 + 1.66103362e+00 -8.99466249e-01 + -2.53947433e+00 8.40084780e-01 + 4.36631397e-01 7.24234261e-02 + -1.87589394e+00 5.08529113e-01 + 4.49563965e+00 -9.43365992e-01 + 1.78876299e+00 -1.27076149e+00 + -1.16269107e-01 -4.55078316e-01 + 1.92966079e+00 -8.05371385e-01 + 2.20632583e+00 -9.00919345e-01 + 1.52387824e+00 -4.82391996e-01 + 8.04004564e-01 -2.73650595e-01 + -7.75326067e-01 1.07469566e+00 + 1.83226282e+00 -4.52173344e-01 + 1.25079758e-01 -3.52895417e-02 + -9.90957437e-01 8.55993130e-01 + 1.71623322e+00 -7.08691667e-01 + -2.86175924e+00 6.75160955e-01 + -8.40817853e-01 -1.00361809e-01 + 1.33393000e+00 -4.65788123e-01 + 5.29394114e-01 -5.44881619e-02 + -8.07435599e-01 8.27353370e-01 + -4.33165824e+00 1.97299638e+00 + 1.26452422e+00 -8.34070486e-01 + 1.45996394e-02 2.97736043e-01 + -1.64489287e+00 6.72839598e-01 + -5.74234578e+00 3.20975117e+00 + 2.13841341e-02 3.64514015e-01 + 6.68084924e+00 -2.27464254e+00 + -3.22881590e+00 8.01879324e-01 + 3.02534313e-01 -4.56222796e-01 + -5.84520734e+00 1.95678162e+00 + 2.81515232e+00 -1.72101318e+00 + -2.39620908e-01 2.69145522e-01 + -7.41669691e-01 -2.30283281e-01 + -2.15682714e+00 3.45313021e-01 + 1.23475788e+00 -7.32276553e-01 + -1.71816113e-01 1.20419560e-02 + 1.89174235e+00 2.27435901e-01 + -3.64511114e-01 1.72260361e-02 + -3.24143860e+00 6.50125817e-01 + -2.25707409e+00 5.66970751e-01 + 1.03901456e+00 -1.00588433e+00 + -5.09159710e+00 1.58736109e+00 + 1.45534075e+00 -5.83787452e-01 + 4.28879587e+00 -1.58006866e+00 + 8.52384427e-01 -1.11042299e+00 + 4.51431615e+00 -2.63844265e+00 + -4.33042648e+00 1.86497078e+00 + -2.13568046e+00 5.82559743e-01 + -4.42568887e+00 1.26131214e+00 + 3.15821315e+00 -1.61515905e+00 + -3.14125204e+00 8.49604386e-01 + 6.54152300e-01 -2.04624711e-01 + -3.73374317e-01 9.94187820e-02 + -3.96177282e+00 1.27245623e+00 + 9.59825199e-01 -1.15547861e+00 + 3.56902055e+00 -1.46591091e+00 + 1.55433633e-02 6.93544345e-01 + 1.15684646e+00 -4.99836352e-01 + 3.11824573e+00 -4.75900506e-01 + -8.61706369e-01 -3.50774059e-01 + 9.89057391e-01 -7.16878802e-01 + -4.94787870e+00 2.09137481e+00 + 1.37777347e+00 -1.34946349e+00 + -1.13161577e+00 8.05114754e-01 + 8.12020675e-01 -1.04849421e+00 + 4.73783881e+00 -2.26718812e+00 + 8.99579366e-01 -8.89764451e-02 + 4.78524868e+00 -2.25795843e+00 + 1.75164590e+00 -1.73822209e-01 + 1.30204590e+00 -7.26724717e-01 + -7.26526403e-01 -5.23925361e-02 + 2.01255351e+00 -1.69965366e+00 + 9.87852740e-01 -4.63577220e-01 + 2.45957762e+00 -1.29278962e+00 + -3.13817948e+00 1.64433038e+00 + -1.76302159e+00 9.62784302e-01 + -1.91106331e+00 5.81460008e-01 + -3.30883001e+00 1.30378978e+00 + 5.54376450e-01 3.78814272e-01 + 1.09982111e+00 -1.47969612e+00 + -2.61300705e-02 -1.42573464e-01 + -2.22096157e+00 7.75684440e-01 + 1.70319323e+00 -2.89738444e-01 + -1.43223842e+00 6.39284281e-01 + 2.34360959e-01 -1.64379268e-01 + -2.67147991e+00 9.46548086e-01 + 1.51131425e+00 -4.91594395e-01 + -2.48446856e+00 1.01286123e+00 + 1.50534658e-01 -2.94620246e-01 + -1.66966792e+00 1.67755508e+00 + -1.50094241e+00 3.30163095e-01 + 2.27681194e+00 -1.08064317e+00 + 2.05122965e+00 -1.15165939e+00 + -4.23509309e-01 -6.56906167e-02 + 1.80084023e+00 -1.07228556e+00 + -2.65769521e+00 1.18023206e+00 + 2.02852676e+00 -8.06793574e-02 + -4.49544185e+00 2.68200163e+00 + -7.50043216e-01 1.17079331e+00 + 6.80060893e-02 3.99055351e-01 + -3.83634635e+00 1.38406887e+00 + 3.24858545e-01 -9.25273218e-02 + -2.19895100e+00 1.47819500e+00 + -3.61569522e-01 -1.03188739e-01 + 1.12180375e-01 -9.52696354e-02 + -1.31477803e+00 1.79900570e-01 + 2.39573628e+00 -6.09739269e-01 + -1.00135700e+00 6.02837296e-01 + -4.11994589e+00 2.49599192e+00 + -1.54196236e-01 -4.84921951e-01 + 5.92569908e-01 -1.87310359e-01 + 3.85407741e+00 -1.50979925e+00 + 5.17802528e+00 -2.26032607e+00 + -1.37018916e+00 1.87111822e-01 + 8.46682996e-01 -3.56676331e-01 + -1.17559949e+00 5.29057734e-02 + -5.56475671e-02 6.79049243e-02 + 1.07851745e+00 -5.14535101e-01 + -2.71622446e+00 1.00151846e+00 + -1.08477208e+00 8.81391054e-01 + 5.50755824e-01 -5.20577727e-02 + 4.70885495e+00 -2.04220397e+00 + -1.87375336e-01 -6.16962830e-02 + 3.52097100e-01 2.21163550e-01 + 7.07929984e-01 -1.75827590e-01 + -1.22149219e+00 1.83084346e-01 + 2.58247412e+00 -6.15914898e-01 + -6.01206182e-01 -2.29832987e-01 + 9.83360449e-01 -3.75870060e-01 + -3.20027685e+00 1.35467480e+00 + 1.79178978e+00 -1.38531981e+00 + -3.30376867e-01 -1.16250192e-01 + -1.89053055e+00 5.68463567e-01 + -4.20604849e+00 1.65429681e+00 + -1.01185529e+00 1.92801240e-01 + -6.18819882e-01 5.42206996e-01 + -5.08091672e+00 2.61598591e+00 + -2.62570344e+00 2.51590658e+00 + 3.05577906e+00 -1.49090609e+00 + 2.77609677e+00 -1.37681378e+00 + -7.93515301e-02 4.28072744e-01 + -2.08359471e+00 8.94334295e-01 + 2.20163801e+00 4.01127167e-02 + -1.18145785e-01 -2.06822464e-01 + -2.74788298e-01 2.96250607e-01 + 1.59613555e+00 -3.87246203e-01 + -3.82971472e-01 -3.39716093e-02 + -4.20311307e-02 3.88529510e-01 + 1.52128574e+00 -9.33138876e-01 + -9.06584458e-01 -2.75016094e-02 + 3.56216834e+00 -9.99384622e-01 + 2.11964220e+00 -9.98749118e-02 + 4.01203480e+00 -2.03032745e+00 + -1.24171557e+00 1.97596725e-01 + -1.57230455e+00 4.14126609e-01 + -1.85484741e+00 5.40041563e-01 + 1.76329831e+00 -6.95967734e-01 + -2.29439232e-01 5.08669245e-01 + -5.45124276e+00 2.26907549e+00 + -5.71364288e-02 5.04476476e-01 + 3.12468018e+00 -1.46358879e+00 + 8.20017359e-01 6.51949028e-01 + -1.33977500e+00 2.83634232e-04 + -1.83311685e+00 1.23947117e+00 + 6.31205922e-01 1.19792164e-02 + -2.21967834e+00 6.94056232e-01 + -1.41693842e+00 9.93526233e-01 + -7.58885703e-01 6.78547347e-01 + 3.60239086e+00 -1.08644935e+00 + 6.72217073e-02 3.00036011e-02 + -3.42680958e-01 -3.48049352e-01 + 1.87546079e+00 -4.78018246e-01 + 7.00485821e-01 -3.52905383e-01 + -8.54580948e-01 8.17330861e-01 + 8.19123706e-01 -5.73927281e-01 + 2.70855639e-01 -3.08940052e-01 + -1.05059952e+00 3.27873168e-01 + 1.08282999e+00 4.84559349e-02 + -7.89899220e-01 1.22291138e+00 + -2.87939816e+00 7.17403497e-01 + -2.08429452e+00 8.87409226e-01 + 1.58409232e+00 -4.74123532e-01 + 1.26882735e+00 1.59162510e-01 + -2.53782993e+00 6.18253491e-01 + -8.92757445e-01 3.35979011e-01 + 1.31867900e+00 -1.17355054e+00 + 1.14918879e-01 -5.35184038e-01 + -1.70288738e-01 5.35868087e-02 + 4.21355121e-01 5.41848690e-02 + 2.07926943e+00 -5.72538144e-01 + 4.08788970e-01 3.77655777e-01 + -3.39631381e+00 9.84216764e-01 + 2.94170163e+00 -1.83120916e+00 + -7.94798752e-01 7.39889052e-01 + 1.46555463e+00 -4.62275563e-01 + 2.57255955e+00 -1.04671434e+00 + 8.45042540e-01 -1.96952892e-01 + -3.23526646e+00 1.60049846e+00 + 3.21948565e+00 -8.88376674e-01 + 1.43005104e+00 -9.21561086e-01 + 8.82360506e-01 2.98403872e-01 + -8.91168097e-01 1.01319072e+00 + -5.13215241e-01 -2.47182649e-01 + -1.35759444e+00 7.07450608e-02 + -4.04550983e+00 2.23534867e+00 + 1.39348883e+00 3.81637747e-01 + -2.85676418e+00 1.53240862e+00 + -1.37183120e+00 6.37977425e-02 + -3.88195859e+00 1.73887145e+00 + 1.19509776e+00 -6.25013512e-01 + -2.80062734e+00 1.79840585e+00 + 1.96558429e+00 -4.70997234e-01 + 1.93111352e+00 -9.70318441e-01 + 3.57991190e+00 -1.65065116e+00 + 2.12831714e+00 -1.11531708e+00 + -3.95661018e-01 -8.54339904e-02 + -2.41630441e+00 1.65166304e+00 + 7.55412624e-01 -1.53453579e-01 + -1.77043450e+00 1.39928715e+00 + -9.32631260e-01 8.73649199e-01 + 1.53342205e+00 -8.39569765e-01 + -6.29846924e-02 1.25023084e-01 + 3.31509049e+00 -1.10733235e+00 + -2.18957109e+00 3.07376993e-01 + -2.35740747e+00 6.47437564e-01 + -2.22142438e+00 8.47318938e-01 + -6.51401147e-01 3.48398562e-01 + 2.75763095e+00 -1.21390708e+00 + 1.12550484e+00 -5.61412847e-01 + -5.65053161e-01 6.74365205e-02 + 1.68952456e+00 -6.57566096e-01 + 8.95598401e-01 3.96738993e-01 + -1.86537066e+00 9.44129208e-01 + -2.59933294e+00 2.57423247e-01 + -6.59598267e-01 1.91828851e-02 + -2.64506676e+00 8.41783205e-01 + -1.25911802e+00 5.52425066e-01 + -1.39754507e+00 3.73689222e-01 + 5.49550729e-02 1.35071215e+00 + 3.31874811e+00 -1.05682424e+00 + 3.63159604e+00 -1.42864695e+00 + -4.45944617e+00 1.42889446e+00 + 5.87314342e-01 -4.88892988e-01 + -7.26130820e-01 1.51936106e-01 + -1.79246441e+00 6.05888105e-01 + -5.50948207e-01 6.21443081e-01 + -3.17246063e-01 1.77213880e-01 + -2.00098937e+00 1.23799074e+00 + 4.33790961e+00 -1.08490465e+00 + -2.03114114e+00 1.31613237e+00 + -6.29216542e+00 1.92406317e+00 + -1.60265624e+00 8.87947500e-01 + 8.64465062e-01 -8.37416270e-01 + -2.14273937e+00 8.05485900e-01 + -2.36844256e+00 6.17915124e-01 + -1.40429636e+00 6.78296866e-01 + 9.99019988e-01 -5.84297572e-01 + 7.38824546e-01 1.68838678e-01 + 1.45681238e+00 3.04641461e-01 + 2.15914949e+00 -3.43089227e-01 + -1.23895930e+00 1.05339864e-01 + -1.23162264e+00 6.46629863e-01 + 2.28183862e+00 -9.24157063e-01 + -4.29615882e-01 5.69130863e-01 + -1.37449121e+00 -9.12032183e-01 + -7.33890904e-01 -3.91865471e-02 + 8.41400661e-01 -4.76002200e-01 + -1.73349274e-01 -6.84143467e-02 + 3.16042891e+00 -1.32651856e+00 + -3.78244609e+00 2.38619718e+00 + -3.69634380e+00 2.22368561e+00 + 1.83766344e+00 -1.65675953e+00 + -1.63206002e+00 1.19484469e+00 + 3.68480064e-01 -5.70764494e-01 + 3.61982479e-01 1.04274409e-01 + 2.48863048e+00 -1.13285542e+00 + -2.81896488e+00 9.47958768e-01 + 5.74952901e-01 -2.75959392e-01 + 3.72783275e-01 -3.48937848e-01 + 1.95935716e+00 -1.06750415e+00 + 5.19357531e+00 -2.32070803e+00 + 4.09246149e+00 -1.89976700e+00 + -3.36666087e-01 8.17645057e-02 + 1.85453493e-01 3.76913151e-01 + -3.06458262e+00 1.34106402e+00 + -3.13796566e+00 7.00485099e-01 + 1.42964058e+00 -1.35536932e-01 + -1.23440423e-01 4.60094177e-02 + -2.86753037e+00 -5.21724160e-02 + 2.67113726e+00 -1.83746924e+00 + -1.35335062e+00 1.28238073e+00 + -2.43569899e+00 1.25998539e+00 + 1.26036740e-01 -2.35416844e-01 + -1.35725745e+00 7.37788491e-01 + -3.80897538e-01 3.30757889e-01 + 6.58694434e-01 -1.07566603e+00 + 2.11273640e+00 -9.02260632e-01 + 4.00755057e-01 -2.49229150e-02 + -1.80095812e+00 9.73099742e-01 + -2.68408372e+00 1.63737364e+00 + -2.66079826e+00 7.47289412e-01 + -9.92321439e-02 -1.49331396e-01 + 4.45678251e+00 -1.80352394e+00 + 1.35962915e+00 -1.31554389e+00 + -7.76601417e-01 -9.66173523e-02 + 1.68096348e+00 -6.27235133e-01 + 1.53081227e-01 -3.54216830e-01 + -1.54913095e+00 3.43689269e-01 + 5.29187357e-02 -6.73916964e-01 + -2.06606084e+00 8.34784242e-01 + 1.73701179e+00 -6.06467340e-01 + 1.55856757e+00 -2.58642780e-01 + 1.04349101e+00 -4.43027348e-01 + -1.02397719e+00 1.01308824e+00 + -2.13860204e-01 -4.73347361e-01 + -2.59004955e+00 1.43367853e+00 + 7.98457679e-01 2.18621627e-02 + -1.32974762e+00 4.61802208e-01 + 3.21419359e-01 2.30723316e-02 + 2.87201888e-02 6.24566672e-02 + -1.22261418e+00 6.02340363e-01 + 1.28750335e+00 -3.34839548e-02 + -9.67952623e-01 4.34470505e-01 + 2.02850324e+00 -9.05160255e-01 + -4.13946010e+00 2.33779091e+00 + -4.47508806e-01 3.06440495e-01 + -3.91543394e+00 1.68251022e+00 + -6.45193001e-01 5.29781162e-01 + -2.15518916e-02 5.07278355e-01 + -2.83356868e+00 1.00670227e+00 + 1.82989749e+00 -1.37329222e+00 + -1.09330213e+00 1.08560688e+00 + 1.90533722e+00 -1.28905879e+00 + 2.33986084e+00 2.30642626e-02 + 8.01940220e-01 -1.63986962e+00 + -4.23415165e+00 2.07530423e+00 + 9.33382522e-01 -7.62917211e-01 + -1.84033954e+00 1.07469401e+00 + -2.81938669e+00 1.07342024e+00 + -7.05169988e-01 2.13124943e-01 + 5.09598137e-01 1.32725493e-01 + -2.34558226e+00 8.62383168e-01 + -1.70322072e+00 2.70893796e-01 + 1.23652660e+00 -7.53216034e-02 + 2.84660646e+00 -3.48178304e-02 + 2.50250128e+00 -1.27770855e+00 + -1.00279469e+00 8.77194218e-01 + -4.34674121e-02 -2.12091350e-01 + -5.84151289e-01 1.50382340e-01 + -1.79024013e+00 4.24972808e-01 + -1.23434666e+00 -8.85546570e-02 + 1.36575412e+00 -6.42639880e-01 + -1.98429947e+00 2.27650336e-01 + 2.36253589e+00 -1.51340773e+00 + 8.79157643e-01 6.84142159e-01 + -2.18577755e+00 2.76526200e-01 + -3.55473434e-01 8.29976561e-01 + 1.16442595e+00 -5.97699411e-01 + -7.35528097e-01 2.40318183e-01 + -1.73702631e-01 7.33788663e-02 + -1.40451745e+00 3.24899628e-01 + -2.05434385e+00 5.68123738e-01 + 8.47876642e-01 -5.74224294e-01 + -6.91955602e-01 1.26009087e+00 + 2.56574498e+00 -1.15602581e+00 + 3.93306545e+00 -1.38398209e+00 + -2.73230251e+00 4.89062581e-01 + -1.04315474e+00 6.06335547e-01 + 1.23231431e+00 -4.46675065e-01 + -3.93035285e+00 1.43287651e+00 + -1.02132111e+00 9.58919791e-01 + -1.49425352e+00 1.06456165e+00 + -6.26485337e-01 1.03791402e+00 + -6.61772998e-01 2.63275425e-01 + -1.80940386e+00 5.70767403e-01 + 9.83720450e-01 -1.39449756e-01 + -2.24619662e+00 9.01044870e-01 + 8.94343014e-01 5.31038678e-02 + 1.95518199e-01 -2.81343295e-01 + -2.30533019e-01 -1.74478106e-01 + -2.01550361e+00 5.55958010e-01 + -4.36281469e+00 1.94374226e+00 + -5.18530457e+00 2.89278357e+00 + 2.67289101e+00 -2.98511449e-01 + -1.53566179e+00 -1.00588944e-01 + -6.09943217e-02 -1.56986047e-01 + -5.22146452e+00 1.66209208e+00 + -3.69777478e+00 2.26154873e+00 + 2.24607181e-01 -4.86934960e-01 + 2.49909450e+00 -1.03033370e+00 + -1.07841120e+00 8.22388054e-01 + -3.20697089e+00 1.09536143e+00 + 3.43524232e+00 -1.47289362e+00 + -5.65784134e-01 4.60365175e-01 + -1.76714734e+00 1.57752346e-01 + -7.77620365e-01 5.60153443e-01 + 6.34399352e-01 -5.22339836e-01 + 2.91011875e+00 -9.72623380e-01 + -1.19286824e+00 6.32370253e-01 + -2.18327609e-01 8.23953181e-01 + 3.42430842e-01 1.37098055e-01 + 1.28658034e+00 -9.11357320e-01 + 2.06914465e+00 -6.67556382e-01 + -6.69451020e-01 -6.38605102e-01 + -2.09312398e+00 1.16743634e+00 + -3.63778357e+00 1.91919157e+00 + 8.74685911e-01 -1.09931208e+00 + -3.91496791e+00 1.00808357e+00 + 1.29621330e+00 -8.32239802e-01 + 9.00222045e-01 -1.31159793e+00 + -1.12242062e+00 1.98517079e-01 + -3.71932852e-01 1.31667093e-01 + -2.23829610e+00 1.26328346e+00 + -2.08365062e+00 9.93385336e-01 + -1.91082720e+00 7.45866855e-01 + 4.38024917e+00 -2.05901118e+00 + -2.28872886e+00 6.85279335e-01 + 1.01274497e-01 -3.26227153e-01 + -5.04447572e-01 -3.18619513e-01 + 1.28537006e+00 -1.04573551e+00 + -7.83175212e-01 1.54791645e-01 + -3.89239175e+00 1.60017929e+00 + -8.87877111e-01 -1.04968005e-01 + 9.32215179e-01 -5.58691113e-01 + -6.44977127e-01 -2.23018375e-01 + 1.10141900e+00 -1.00666432e+00 + 2.92755687e-01 -1.45480350e-01 + 7.73580681e-01 -2.21150567e-01 + -1.40873709e+00 7.61548044e-01 + -8.89031805e-01 -3.48542923e-01 + 4.16844267e-01 -2.39914494e-01 + -4.64265832e-01 7.29581138e-01 + 1.99835179e+00 -7.70542813e-01 + 4.20523191e-02 -2.18783563e-01 + -6.32611758e-01 -3.09926115e-01 + 6.82912198e-02 -8.48327050e-01 + 1.92425229e+00 -1.37876951e+00 + 3.49461782e+00 -1.88354255e+00 + -3.25209026e+00 1.49809395e+00 + 6.59273182e-01 -2.37435654e-01 + -1.15517300e+00 8.46134387e-01 + 1.26756151e+00 -4.58988026e-01 + -3.99178418e+00 2.04153008e+00 + 7.05687841e-01 -6.83433306e-01 + -1.61997342e+00 8.16577004e-01 + -3.89750399e-01 4.29753250e-01 + -2.53026432e-01 4.92861432e-01 + -3.16788324e+00 4.44285524e-01 + -7.86248901e-01 1.12753716e+00 + -3.02351433e+00 1.28419015e+00 + -1.30131355e+00 1.71226678e+00 + -4.08843475e+00 1.62063214e+00 + -3.09209403e+00 1.19958520e+00 + 1.49102271e+00 -1.11834864e+00 + -3.18059348e+00 5.74587042e-01 + 2.06054867e+00 3.25797860e-03 + -3.50999200e+00 2.02412428e+00 + -8.26610023e-01 3.46528211e-01 + 2.00546034e+00 -4.07333110e-01 + -9.69941653e-01 4.80953753e-01 + 4.47925660e+00 -2.33127314e+00 + 2.03845790e+00 -9.90439915e-01 + -1.11349191e+00 4.31183918e-01 + -4.03628396e+00 1.68509679e+00 + -1.48177601e+00 7.74322088e-01 + 3.07369385e+00 -9.57465886e-01 + 2.39011286e+00 -6.44506921e-01 + 2.91561991e+00 -8.78627328e-01 + 1.10212733e+00 -4.21637388e-01 + 5.31985231e-01 -6.17445696e-01 + -6.82340929e-01 -2.93529716e-01 + 1.94290679e+00 -4.64268634e-01 + 1.92262116e+00 -7.93142835e-01 + 4.73762800e+00 -1.63654174e+00 + -3.17848641e+00 8.05791391e-01 + 4.08739432e+00 -1.80816807e+00 + -7.60648826e-01 1.24216138e-01 + -2.24716400e+00 7.90020937e-01 + 1.64284052e+00 -7.18784070e-01 + 1.04410012e-01 -7.11195880e-02 + 2.18268225e+00 -7.01767831e-01 + 2.06218013e+00 -8.70251746e-01 + -1.35266581e+00 7.08456358e-01 + -1.38157779e+00 5.14401086e-01 + -3.28326008e+00 1.20988399e+00 + 8.85358917e-01 -8.12213495e-01 + -2.34067500e+00 3.67657353e-01 + 3.96878127e+00 -1.66841450e+00 + 1.36518053e+00 -8.33436812e-01 + 5.25771988e-01 -5.06121987e-01 + -2.25948361e+00 1.30663765e+00 + -2.57662070e+00 6.32114628e-01 + -3.43134685e+00 2.38106008e+00 + 2.31571924e+00 -1.56566818e+00 + -2.95397202e+00 1.05661888e+00 + -1.35331242e+00 6.76383411e-01 + 1.40977132e+00 -1.17775938e+00 + 1.52561996e+00 -9.83147176e-01 + 2.26550832e+00 -2.10464123e-02 + 6.23371684e-01 -5.30768122e-01 + -4.42356624e-01 9.72226986e-01 + 2.31517901e+00 -1.08468105e+00 + 1.97236640e+00 -1.42016619e+00 + 3.18618687e+00 -1.45056343e+00 + -2.75880360e+00 5.40254980e-01 + -1.92916581e+00 1.45029864e-01 + 1.90022524e+00 -6.03805754e-01 + -1.05446211e+00 5.74361752e-01 + 1.45990390e+00 -9.28233993e-01 + 5.14960557e+00 -2.07564096e+00 + -7.53104842e-01 1.55876958e-01 + 8.09490983e-02 -8.58886384e-02 + -1.56894969e+00 4.53497227e-01 + 1.36944658e-01 5.60670875e-01 + -5.32635329e-01 4.40309945e-01 + 1.32507853e+00 -5.83670099e-01 + 1.20676031e+00 -8.02296831e-01 + -3.65023422e+00 1.17211368e+00 + 1.53393850e+00 -6.17771312e-01 + -3.99977129e+00 1.71415137e+00 + 5.70705058e-01 -4.60771539e-01 + -2.20608002e+00 1.07866596e+00 + -1.09040244e+00 6.77441076e-01 + -5.09886482e-01 -1.97282128e-01 + -1.58062785e+00 6.18333697e-01 + -1.53295020e+00 4.02168701e-01 + -5.18580598e-01 2.25767177e-01 + 1.59514316e+00 -2.54983617e-01 + -5.91938655e+00 2.68223782e+00 + 2.84200509e+00 -1.04685313e+00 + 1.31298664e+00 -1.16672614e+00 + -2.36660033e+00 1.81359460e+00 + 6.94163290e-02 3.76658816e-01 + 2.33973934e+00 -8.33173023e-01 + -8.24640389e-01 7.83717285e-01 + -1.02888281e+00 1.04680766e+00 + 1.34750745e+00 -5.89568160e-01 + -2.48761231e+00 7.44199284e-01 + -1.04501559e+00 4.72326911e-01 + -3.14610089e+00 1.89843692e+00 + 2.13003416e-01 5.76633620e-01 + -1.69239608e+00 5.66070021e-01 + 1.80491280e+00 -9.31701080e-01 + -6.94362572e-02 6.96026587e-01 + 1.36502578e+00 -6.85599000e-02 + -7.76764337e-01 3.64328661e-01 + -2.67322167e+00 6.80150021e-01 + 1.84338485e+00 -1.18487494e+00 + 2.88009231e+00 -1.25700411e+00 + 1.17114433e+00 -7.69727080e-01 + 2.11576167e+00 2.81502116e-01 + -1.51470088e+00 2.61553540e-01 + 1.18923669e-01 -1.17890202e-01 + 4.48359786e+00 -1.81427466e+00 + -1.27055948e+00 9.92388998e-01 + -8.00276606e-01 9.11326621e-02 + 7.51764024e-01 -1.03676498e-01 + 1.35769348e-01 -2.11470084e-01 + 2.50731332e+00 -1.12418270e+00 + -2.49752781e-01 7.81224033e-02 + -6.23037902e-01 3.16599691e-01 + -3.93772902e+00 1.37195391e+00 + 1.74256361e+00 -1.12363582e+00 + -1.49737281e+00 5.98828310e-01 + 7.75592115e-01 -4.64733802e-01 + -2.26027693e+00 1.36991118e+00 + -1.62849836e+00 7.36899107e-01 + 2.36850751e+00 -9.32126872e-01 + 5.86169745e+00 -2.49342512e+00 + -5.37092226e-01 1.23821274e+00 + 2.80535867e+00 -1.93363302e+00 + -1.77638106e+00 9.10050276e-01 + 3.02692018e+00 -1.60774676e+00 + 1.97833084e+00 -1.50636531e+00 + 9.09168906e-01 -8.83799359e-01 + 2.39769655e+00 -7.56977869e-01 + 1.47283981e+00 -1.06749890e+00 + 2.92060943e-01 -6.07040605e-01 + -2.09278201e+00 7.71858590e-01 + 7.10015905e-01 -5.42768432e-01 + -2.16826169e-01 1.56897896e-01 + 4.56288247e+00 -2.08912680e+00 + -6.63374020e-01 6.67325183e-01 + 1.80564442e+00 -9.76366134e-01 + 3.28720168e+00 -4.66575145e-01 + -1.60463695e-01 -2.58428153e-01 + 1.78590750e+00 -3.96427146e-01 + 2.75950306e+00 -1.82102856e+00 + -1.18234310e+00 6.28073320e-01 + 4.11415835e+00 -2.33551216e+00 + 1.38721004e+00 -2.77450622e-01 + -2.94903545e+00 1.74813352e+00 + 8.67290400e-01 -6.51667894e-01 + 2.70022274e+00 -8.11832480e-01 + -2.06766146e+00 8.24047249e-01 + 3.90717142e+00 -1.20155758e+00 + -2.95102809e+00 1.36667968e+00 + 6.08815147e+00 -2.60737974e+00 + 2.78576476e+00 -7.86628755e-01 + -3.26258407e+00 1.09302450e+00 + 1.59849422e+00 -1.09705202e+00 + -2.50600710e-01 1.63243175e-01 + -4.90477087e-01 -4.57729572e-01 + -1.24837181e+00 3.22157840e-01 + -2.46341049e+00 1.06517849e+00 + 9.62880751e-01 4.56962496e-01 + 3.99964487e-01 2.07472802e-01 + 6.36657705e-01 -3.46400942e-02 + 4.91231407e-02 -1.40289235e-02 + -4.66683524e-02 -3.72326100e-01 + -5.22049702e-01 -1.70440260e-01 + 5.27062938e-01 -2.32628395e-01 + -2.69440318e+00 1.18914874e+00 + 3.65087539e+00 -1.53427267e+00 + -1.16546364e-01 4.93245392e-02 + 7.55931384e-01 -3.02980139e-01 + 2.06338745e+00 -6.24841225e-01 + 1.31177908e-01 7.29338183e-01 + 1.48021784e+00 -6.39509896e-01 + -5.98656707e-01 2.84525503e-01 + -2.18611080e+00 1.79549812e+00 + -2.91673624e+00 2.15772237e-01 + -8.95591350e-01 7.68250538e-01 + 1.36139762e+00 -1.93845144e-01 + 5.45730414e+00 -2.28114404e+00 + 3.22747247e-01 9.33582332e-01 + -1.46384504e+00 1.12801186e-01 + 4.26728166e-01 -2.33481242e-01 + -1.41327270e+00 8.16103740e-01 + -2.53998067e-01 1.44906646e-01 + -1.32436467e+00 1.87556361e-01 + -3.77313086e+00 1.32896038e+00 + 3.77651731e+00 -1.76548043e+00 + -2.45297093e+00 1.32571926e+00 + -6.55900588e-01 3.56921462e-01 + 9.25558722e-01 -4.51988954e-01 + 1.20732231e+00 -3.02821614e-01 + 3.72660154e-01 -1.89365208e-01 + -1.77090939e+00 9.18087975e-01 + 3.01127567e-01 2.67965829e-01 + -1.76708900e+00 4.62069259e-01 + -2.71812099e+00 1.57233508e+00 + -5.35297633e-01 4.99231535e-01 + 1.50507631e+00 -9.85763646e-01 + 3.00424787e+00 -1.29837562e+00 + -4.99311105e-01 3.91086482e-01 + 1.30125207e+00 -1.26247924e-01 + 4.01699483e-01 -4.46909391e-01 + -1.33635257e+00 5.12068703e-01 + 1.39229757e+00 -9.10974858e-01 + -1.74229508e+00 1.49475978e+00 + -1.21489414e+00 4.04193753e-01 + -3.36537605e-01 -6.74335427e-01 + -2.79186828e-01 8.48314720e-01 + -2.03080140e+00 1.66599815e+00 + -3.53064281e-01 -7.68582906e-04 + -5.30305657e+00 2.91091546e+00 + -1.20049972e+00 8.26578358e-01 + 2.95906989e-01 2.40215920e-01 + -1.42955534e+00 4.63480310e-01 + -1.87856619e+00 8.21459385e-01 + -2.71124720e+00 1.80246843e+00 + -3.06933780e+00 1.22235760e+00 + 5.21935582e-01 -1.27298218e+00 + -1.34175797e+00 7.69018937e-01 + -1.81962785e+00 1.15528991e+00 + -3.99227550e-01 2.93821598e-01 + 1.22533179e+00 -4.73846323e-01 + -2.08068359e-01 -1.75039817e-01 + -2.03068526e+00 1.50370503e+00 + -3.27606113e+00 1.74906330e+00 + -4.37802587e-01 -2.26956048e-01 + -7.69774213e-02 -3.54922468e-01 + 6.47160749e-02 -2.07334721e-01 + -1.37791524e+00 4.43766709e-01 + 3.29846803e+00 -1.04060799e+00 + -3.63704046e+00 1.05800226e+00 + -1.26716116e+00 1.13077353e+00 + 1.98549075e+00 -1.31864807e+00 + 1.85159500e+00 -5.78629560e-01 + -1.55295206e+00 1.23655857e+00 + 6.76026255e-01 9.18824125e-02 + 1.23418960e+00 -4.68162027e-01 + 2.43186642e+00 -9.22422440e-01 + -3.18729701e+00 1.77582673e+00 + -4.02945613e+00 1.14303496e+00 + -1.92694576e-01 1.03301431e-01 + 1.89554730e+00 -4.60128096e-01 + -2.55626581e+00 1.16057084e+00 + 6.89144365e-01 -9.94982900e-01 + -4.44680606e+00 2.19751983e+00 + -3.15196193e+00 1.18762993e+00 + -1.17434977e+00 1.04534656e+00 + 8.58386984e-02 -1.03947487e+00 + 3.33354973e-01 5.54813610e-01 + -9.37631808e-01 3.33450150e-01 + -2.50232471e+00 5.39720635e-01 + 1.03611949e+00 -7.16304095e-02 + -2.05556816e-02 -3.28992265e-01 + -2.24176201e+00 1.13077506e+00 + 4.53583688e+00 -1.10710212e+00 + 4.77389762e-01 -8.99445512e-01 + -2.69075551e+00 6.83176866e-01 + -2.21779724e+00 1.16916849e+00 + -1.09669056e+00 2.10044765e-01 + -8.45367920e-01 -8.45951423e-02 + 4.37558941e-01 -6.95904256e-01 + 1.84884195e+00 -1.71205136e-01 + -8.36371957e-01 5.62862478e-01 + 1.27786531e+00 -1.33362147e+00 + 2.90684492e+00 -7.49892184e-01 + -3.38652716e+00 1.51180670e+00 + -1.30945978e+00 7.09261928e-01 + -7.50471924e-01 -5.24637889e-01 + 1.18580718e+00 -9.97943971e-04 + -7.55395645e+00 3.19273590e+00 + 1.72822535e+00 -1.20996962e+00 + 5.67374320e-01 6.19573416e-01 + -2.99163781e+00 1.79721534e+00 + 1.49862187e+00 -6.05631846e-02 + 1.79503506e+00 -4.90419706e-01 + 3.85626054e+00 -1.95396324e+00 + -9.39188410e-01 7.96498057e-01 + 2.91986664e+00 -1.29392724e+00 + -1.54265750e+00 6.40727933e-01 + 1.14919794e+00 1.20834257e-01 + 2.00936817e+00 -1.53728359e+00 + 3.72468420e+00 -1.38704612e+00 + -1.27794802e+00 3.48543179e-01 + 3.63294077e-01 5.70623314e-01 + 1.49381016e+00 -6.04500534e-01 + 2.98912256e+00 -1.72295726e+00 + -1.80833817e+00 2.94907625e-01 + -3.19669622e+00 1.31888700e+00 + 1.45889401e+00 -8.88448639e-01 + -2.80045388e+00 1.01207060e+00 + -4.78379567e+00 1.48646520e+00 + 2.25510003e+00 -7.13372461e-01 + -9.74441433e-02 -2.17766373e-01 + 2.64468496e-01 -3.60842698e-01 + -5.98821713e+00 3.20197892e+00 + 2.67030213e-01 -5.36386416e-01 + 2.24546960e+00 -8.13464649e-01 + -4.89171414e-01 3.86255031e-01 + -7.45713706e-01 6.29800380e-01 + -3.30460503e-01 3.85127284e-01 + -4.19588147e+00 1.52793198e+00 + 5.42078582e-01 -2.61642741e-02 + 4.24938513e-01 -5.72936751e-01 + 2.82717288e+00 -6.75355024e-01 + -1.44741788e+00 5.03578028e-01 + -1.65547573e+00 7.76444277e-01 + 2.20361170e+00 -1.40835680e+00 + -3.69540235e+00 2.32953767e+00 + -1.41909357e-01 2.28989778e-01 + 1.92838879e+00 -8.72525737e-01 + 1.40708100e+00 -6.81849638e-02 + 1.24988112e+00 -1.39470590e-01 + -2.39435855e+00 7.26587655e-01 + 7.03985028e-01 4.85403277e-02 + 4.05214529e+00 -9.16928318e-01 + 3.74198837e-01 -5.04192358e-01 + -8.43374127e-01 2.36064018e-01 + -3.32253349e-01 7.47840055e-01 + -6.03725210e+00 1.95173337e+00 + 4.60829865e+00 -1.51191309e+00 + -1.46247098e+00 1.11140916e+00 + -9.60111157e-01 -1.23189114e-01 + -7.49613187e-01 4.53614129e-01 + -5.77838219e-01 2.07366469e-02 + 8.07652950e-01 -5.16272662e-01 + -6.02556049e-01 5.05318649e-01 + -1.28712445e-01 2.57836512e-01 + -5.27662820e+00 2.11790737e+00 + 5.40819308e+00 -2.15366022e+00 + 9.37742513e-02 -1.60221751e-01 + 4.55902865e+00 -1.24646307e+00 + -9.06582589e-01 1.92928110e-01 + 2.99928996e+00 -8.04301218e-01 + -3.24317381e+00 1.80076061e+00 + 3.20421743e-01 8.76524679e-01 + -5.29606705e-01 -3.16717696e-01 + -1.77264560e+00 7.52686776e-01 + -1.51706824e+00 8.43755103e-01 + 1.52759111e+00 -7.86814243e-01 + 4.74845617e-01 4.21319700e-01 + 6.97829149e-01 -8.15664881e-01 + 3.09564973e+00 -1.06202469e+00 + 2.95320379e+00 -1.98963943e+00 + -4.23033224e+00 1.41013338e+00 + 1.48576206e+00 8.02908511e-02 + 4.52041627e+00 -2.04620399e+00 + 6.58403922e-01 -7.60781799e-01 + 2.10667543e-01 1.15241731e-01 + 1.77702583e+00 -8.10271859e-01 + 2.41277385e+00 -1.46972042e+00 + 1.50685525e+00 -1.99272545e-01 + 7.61665522e-01 -4.11276152e-01 + 1.18352312e+00 -9.59908608e-01 + -3.32031305e-01 8.07500132e-02 + 1.16813118e+00 -1.73095194e-01 + 1.18363346e+00 -5.41565052e-01 + 5.17702179e-01 -7.62442035e-01 + 4.57401006e-01 -1.45951115e-02 + 1.49377115e-01 2.99571605e-01 + 1.40399453e+00 -1.30160353e+00 + 5.26231567e-01 3.52783752e-01 + -1.91136514e+00 4.24228635e-01 + 1.74156701e+00 -9.92076776e-01 + -4.89323391e+00 2.32483507e+00 + 2.54011209e+00 -8.80366295e-01 + -5.56925706e-01 1.48842026e-01 + -2.35904668e+00 9.60474853e-01 + 1.42216971e+00 -4.67062761e-01 + -1.10809680e+00 7.68684300e-01 + 4.09674726e+00 -1.90795680e+00 + -2.23048923e+00 9.03812542e-01 + 6.57025763e-01 1.36514871e-01 + 2.10944145e+00 -9.78897838e-02 + 1.22552525e+00 -2.50303867e-01 + 2.84620103e-01 -5.30164020e-01 + -2.13562585e+00 1.03503056e+00 + 1.32414902e-01 -8.14190240e-03 + -5.82433561e-01 3.21020292e-01 + -5.06473247e-01 3.11530419e-01 + 1.57162465e+00 -1.20763919e+00 + -1.43155284e+00 -2.51203698e-02 + -1.47093713e+00 -1.39620999e-01 + -2.65765643e+00 1.06091403e+00 + 2.45992927e+00 -5.88815836e-01 + -1.28440162e+00 -1.99377398e-01 + 6.11257504e-01 -3.73577401e-01 + -3.46606103e-01 6.06081290e-01 + 3.76687505e+00 -8.80181424e-01 + -1.03725103e+00 1.45177517e+00 + 2.76659936e+00 -1.09361320e+00 + -3.61311296e+00 9.75032455e-01 + 3.22878655e+00 -9.69497365e-01 + 1.43560379e+00 -5.52524585e-01 + 2.94042153e+00 -1.79747037e+00 + 1.30739580e+00 2.47989248e-01 + -4.05056982e-01 1.22831715e+00 + -2.25827421e+00 2.30604626e-01 + 3.69262926e-01 4.32714650e-02 + -5.52064063e-01 6.07806340e-01 + 7.03325987e+00 -2.17956730e+00 + -2.37823835e-01 -8.28068639e-01 + -4.84279888e-01 5.67765194e-01 + -3.15863410e+00 1.02241617e+00 + -3.39561593e+00 1.36876374e+00 + -2.78482934e+00 6.81641104e-01 + -4.37604334e+00 2.23826340e+00 + -2.54049692e+00 8.22676745e-01 + 3.73264822e+00 -9.93498732e-01 + -3.49536064e+00 1.84771519e+00 + 9.81801604e-01 -5.21278776e-01 + 1.52996831e+00 -1.27386206e+00 + -9.23490293e-01 5.29099482e-01 + -2.76999461e+00 9.24831872e-01 + -3.30029834e-01 -2.49645555e-01 + -1.71156166e+00 5.44940854e-01 + -2.37009487e+00 5.83826982e-01 + -3.03216865e+00 1.04922722e+00 + -2.19539936e+00 1.37558730e+00 + 1.15350207e+00 -6.15318535e-01 + 4.62011792e+00 -2.46714517e+00 + 1.52627952e-02 -1.00618283e-01 + -1.10399342e+00 4.87413533e-01 + 3.55448194e+00 -9.10394190e-01 + -5.21890321e+00 2.44710745e+00 + 1.54289749e+00 -6.54269311e-01 + 2.67935674e+00 -9.92758863e-01 + 1.05801310e+00 2.60054285e-02 + 1.52509097e+00 -4.08768600e-01 + 3.27576917e+00 -1.28769406e+00 + 1.71008412e-01 -2.68739994e-01 + -9.83351344e-04 7.02495897e-02 + -7.60795056e-03 1.61968285e-01 + -1.80620472e+00 4.24934471e-01 + 2.32023297e-02 -2.57284559e-01 + 3.98219478e-01 -4.65361935e-01 + 6.63476988e-01 -3.29823196e-02 + 4.00154707e+00 -1.01792211e+00 + -1.50286870e+00 9.46875359e-01 + -2.22717585e+00 7.50636195e-01 + -3.47381508e-01 -6.51596975e-01 + 2.08076453e+00 -8.22800165e-01 + 2.05099963e+00 -4.00868250e-01 + 3.52576988e-02 -2.54418565e-01 + 1.57342042e+00 -7.62166492e-02 + -1.47019722e+00 3.40861172e-01 + -1.21156090e+00 3.21891246e-01 + 3.79729047e+00 -1.54350764e+00 + 1.26459678e-02 6.99203693e-01 + 1.53974177e-01 4.68643204e-01 + -1.73923561e-01 -1.26229768e-01 + 4.54644993e+00 -2.13951783e+00 + 1.46022547e-01 -4.57084165e-01 + 6.50048037e+00 -2.78872609e+00 + -1.51934912e+00 1.03216768e+00 + -3.06483575e+00 1.81101446e+00 + -2.38212125e+00 9.19559042e-01 + -1.81319611e+00 8.10545112e-01 + 1.70951294e+00 -6.10712680e-01 + 1.67974156e+00 -1.51241453e+00 + -5.94795113e+00 2.56893813e+00 + 3.62633110e-01 -7.46965304e-01 + -2.44042594e+00 8.52761797e-01 + 3.32412550e+00 -1.28439899e+00 + 4.74860766e+00 -1.72821964e+00 + 1.29072541e+00 -8.24872902e-01 + -1.69450702e+00 4.09600876e-01 + 1.29705411e+00 1.22300809e-01 + -2.63597613e+00 8.55612913e-01 + 9.28467301e-01 -2.63550114e-02 + 2.44670264e+00 -4.10123002e-01 + 1.06408206e+00 -5.03361942e-01 + 5.12384049e-02 -1.27116595e-02 + -1.06731272e+00 -1.76205029e-01 + -9.45454582e-01 3.74404917e-01 + 2.54343689e+00 -7.13810545e-01 + -2.54460335e+00 1.31590265e+00 + 1.89864233e+00 -3.98436339e-01 + -1.93990133e+00 6.01474630e-01 + -1.35938824e+00 4.00751788e-01 + 2.38567018e+00 -6.13904880e-01 + 2.18748050e-01 2.62631712e-01 + -2.01388788e+00 1.41474031e+00 + 2.74014581e+00 -1.27448105e+00 + -2.13828583e+00 1.13616144e+00 + 5.98730932e+00 -2.53430080e+00 + -1.72872795e+00 1.53702057e+00 + -2.53263962e+00 1.27342410e+00 + 1.34326968e+00 -1.99395088e-01 + 3.83352666e-01 -1.25683065e-01 + -2.35630657e+00 5.54116983e-01 + -1.94900838e+00 5.76270178e-01 + -1.36699108e+00 -3.40904824e-01 + -2.34727346e+00 -1.93054940e-02 + -3.82779777e+00 1.83025664e+00 + -4.31602080e+00 9.21605705e-01 + 5.54098133e-01 2.33991419e-01 + -4.53591188e+00 1.99833353e+00 + -3.92715909e+00 1.83231482e+00 + 3.91344440e-01 -1.11355111e-01 + 3.48576363e+00 -1.41379449e+00 + -1.42858690e+00 3.84532286e-01 + 1.79519859e+00 -9.23486448e-01 + 8.49691242e-01 -1.76551084e-01 + 1.53618138e+00 8.23835015e-02 + 5.91476520e-02 3.88296940e-02 + 1.44837346e+00 -7.24097604e-01 + -6.79008418e-01 4.04078097e-01 + 2.87555510e+00 -9.51825076e-01 + -1.12379101e+00 2.93457714e-01 + 1.45263980e+00 -6.01960544e-01 + -2.55741621e-01 9.26233518e-01 + 3.54570714e+00 -1.41521877e+00 + -1.61542388e+00 6.57844512e-01 + -3.22844269e-01 3.02823546e-01 + 1.03523913e+00 -6.92730711e-01 + 1.11084909e+00 -3.50823642e-01 + 3.41268693e+00 -1.90865862e+00 + 7.67062858e-01 -9.48792160e-01 + -5.49798016e+00 1.71139960e+00 + 1.14865798e+00 -6.12669150e-01 + -2.18256680e+00 7.78634462e-01 + 4.78857389e+00 -2.55555085e+00 + -1.85555569e+00 8.04311615e-01 + -4.22278799e+00 2.01162524e+00 + -1.56556149e+00 1.54353907e+00 + -3.11527864e+00 1.65973526e+00 + 2.66342611e+00 -1.20449402e+00 + 1.57635314e+00 -1.48716308e-01 + -6.35606865e-01 2.59701180e-01 + 1.02431976e+00 -6.76929904e-01 + 1.12973772e+00 1.49473892e-02 + -9.12758116e-01 2.21533933e-01 + -2.98014470e+00 1.71651189e+00 + 2.74016965e+00 -9.47893923e-01 + -3.47830591e+00 1.34941430e+00 + 1.74757562e+00 -3.72503752e-01 + 5.55820383e-01 -6.47992466e-01 + -1.19871928e+00 9.82429151e-01 + -2.53040133e+00 2.10671307e+00 + -1.94085605e+00 1.38938137e+00 diff --git a/data/mllib/sample_fpgrowth.txt b/data/mllib/sample_fpgrowth.txt new file mode 100644 index 0000000000000..c451583e51317 --- /dev/null +++ b/data/mllib/sample_fpgrowth.txt @@ -0,0 +1,6 @@ +r z h k p +z y x w v u t s +s x o n r +x z y m t s q e +z +x z y r q t p diff --git a/data/mllib/sample_isotonic_regression_data.txt b/data/mllib/sample_isotonic_regression_data.txt new file mode 100644 index 0000000000000..d257b509d4d37 --- /dev/null +++ b/data/mllib/sample_isotonic_regression_data.txt @@ -0,0 +1,100 @@ +0.24579296,0.01 +0.28505864,0.02 +0.31208567,0.03 +0.35900051,0.04 +0.35747068,0.05 +0.16675166,0.06 +0.17491076,0.07 +0.04181540,0.08 +0.04793473,0.09 +0.03926568,0.10 +0.12952575,0.11 +0.00000000,0.12 +0.01376849,0.13 +0.13105558,0.14 +0.08873024,0.15 +0.12595614,0.16 +0.15247323,0.17 +0.25956145,0.18 +0.20040796,0.19 +0.19581846,0.20 +0.15757267,0.21 +0.13717491,0.22 +0.19020908,0.23 +0.19581846,0.24 +0.20091790,0.25 +0.16879143,0.26 +0.18510964,0.27 +0.20040796,0.28 +0.29576747,0.29 +0.43396226,0.30 +0.53391127,0.31 +0.52116267,0.32 +0.48546660,0.33 +0.49209587,0.34 +0.54156043,0.35 +0.59765426,0.36 +0.56144824,0.37 +0.58592555,0.38 +0.52983172,0.39 +0.50178480,0.40 +0.52626211,0.41 +0.58286588,0.42 +0.64660887,0.43 +0.68077511,0.44 +0.74298827,0.45 +0.64864865,0.46 +0.67261601,0.47 +0.65782764,0.48 +0.69811321,0.49 +0.63029067,0.50 +0.61601224,0.51 +0.63233044,0.52 +0.65323814,0.53 +0.65323814,0.54 +0.67363590,0.55 +0.67006629,0.56 +0.51555329,0.57 +0.50892402,0.58 +0.33299337,0.59 +0.36206017,0.60 +0.43090260,0.61 +0.45996940,0.62 +0.56348802,0.63 +0.54920959,0.64 +0.48393677,0.65 +0.48495665,0.66 +0.46965834,0.67 +0.45181030,0.68 +0.45843957,0.69 +0.47118817,0.70 +0.51555329,0.71 +0.58031617,0.72 +0.55481897,0.73 +0.56297807,0.74 +0.56603774,0.75 +0.57929628,0.76 +0.64762876,0.77 +0.66241713,0.78 +0.69301377,0.79 +0.65119837,0.80 +0.68332483,0.81 +0.66598674,0.82 +0.73890872,0.83 +0.73992861,0.84 +0.84242733,0.85 +0.91330954,0.86 +0.88016318,0.87 +0.90719021,0.88 +0.93115757,0.89 +0.93115757,0.90 +0.91942886,0.91 +0.92911780,0.92 +0.95665477,0.93 +0.95002550,0.94 +0.96940337,0.95 +1.00000000,0.96 +0.89801122,0.97 +0.90311066,0.98 +0.90362060,0.99 +0.83477817,1.0 \ No newline at end of file diff --git a/data/mllib/sample_lda_data.txt b/data/mllib/sample_lda_data.txt new file mode 100644 index 0000000000000..2e76702ca9d67 --- /dev/null +++ b/data/mllib/sample_lda_data.txt @@ -0,0 +1,12 @@ +1 2 6 0 2 3 1 1 0 0 3 +1 3 0 1 3 0 0 2 0 0 1 +1 4 1 0 0 4 9 0 1 2 0 +2 1 0 3 0 0 5 0 2 3 9 +3 1 1 9 3 0 2 0 0 1 3 +4 2 0 3 4 5 1 1 1 4 0 +2 1 0 3 0 0 5 0 2 2 9 +1 1 1 9 2 1 2 0 0 1 3 +4 4 0 3 4 2 1 3 0 0 0 +2 8 2 0 3 0 2 0 2 7 2 +1 1 1 9 0 2 2 0 0 3 3 +4 1 0 0 4 5 1 3 0 1 0 diff --git a/dev/change-version-to-2.10.sh b/dev/change-version-to-2.10.sh index 7473c20d28e09..15e0c73b4295e 100755 --- a/dev/change-version-to-2.10.sh +++ b/dev/change-version-to-2.10.sh @@ -16,5 +16,11 @@ # See the License for the specific language governing permissions and # limitations under the License. # + +# Note that this will not necessarily work as intended with non-GNU sed (e.g. OS X) + find . -name 'pom.xml' | grep -v target \ - | xargs -I {} sed -i -e 's|\(artifactId.*\)_2.11|\1_2.10|g' {} + | xargs -I {} sed -i -e 's/\(artifactId.*\)_2.11/\1_2.10/g' {} + +# Also update in parent POM +sed -i -e '0,/ 2.112.10' pom.xml diff --git a/dev/change-version-to-2.11.sh b/dev/change-version-to-2.11.sh index 3957a9f3ba258..c0a8cb4f825c7 100755 --- a/dev/change-version-to-2.11.sh +++ b/dev/change-version-to-2.11.sh @@ -17,5 +17,10 @@ # limitations under the License. # +# Note that this will not necessarily work as intended with non-GNU sed (e.g. OS X) + find . -name 'pom.xml' | grep -v target \ - | xargs -I {} sed -i -e 's|\(artifactId.*\)_2.10|\1_2.11|g' {} + | xargs -I {} sed -i -e 's/\(artifactId.*\)_2.10/\1_2.11/g' {} + +# Also update in parent POM +sed -i -e '0,/ 2.102.11' pom.xml diff --git a/dev/check-license b/dev/check-license index a006f65710d6d..39943f882b6ca 100755 --- a/dev/check-license +++ b/dev/check-license @@ -31,7 +31,7 @@ acquire_rat_jar () { printf "Attempting to fetch rat\n" JAR_DL="${JAR}.part" if [ $(command -v curl) ]; then - curl --silent "${URL}" > "$JAR_DL" && mv "$JAR_DL" "$JAR" + curl -L --silent "${URL}" > "$JAR_DL" && mv "$JAR_DL" "$JAR" elif [ $(command -v wget) ]; then wget --quiet ${URL} -O "$JAR_DL" && mv "$JAR_DL" "$JAR" else diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index b2a7e092a0291..b5a67dd783b93 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -22,8 +22,9 @@ # Expects to be run in a totally empty directory. # # Options: -# --package-only only packages an existing release candidate -# +# --skip-create-release Assume the desired release tag already exists +# --skip-publish Do not publish to Maven central +# --skip-package Do not package and upload binary artifacts # Would be nice to add: # - Send output to stderr and have useful logging in stdout @@ -33,6 +34,9 @@ ASF_PASSWORD=${ASF_PASSWORD:-XXX} GPG_PASSPHRASE=${GPG_PASSPHRASE:-XXX} GIT_BRANCH=${GIT_BRANCH:-branch-1.0} RELEASE_VERSION=${RELEASE_VERSION:-1.2.0} +# Allows publishing under a different version identifier than +# was present in the actual release sources (e.g. rc-X) +PUBLISH_VERSION=${PUBLISH_VERSION:-$RELEASE_VERSION} NEXT_VERSION=${NEXT_VERSION:-1.2.1} RC_NAME=${RC_NAME:-rc2} @@ -51,7 +55,7 @@ set -e GIT_TAG=v$RELEASE_VERSION-$RC_NAME -if [[ ! "$@" =~ --package-only ]]; then +if [[ ! "$@" =~ --skip-create-release ]]; then echo "Creating release commit and publishing to Apache repository" # Artifact publishing git clone https://$ASF_USERNAME:$ASF_PASSWORD@git-wip-us.apache.org/repos/asf/spark.git \ @@ -87,12 +91,25 @@ if [[ ! "$@" =~ --package-only ]]; then git commit -a -m "Preparing development version $next_ver" git push origin $GIT_TAG git push origin HEAD:$GIT_BRANCH - git checkout -f $GIT_TAG + popd + rm -rf spark +fi + +if [[ ! "$@" =~ --skip-publish ]]; then + git clone https://$ASF_USERNAME:$ASF_PASSWORD@git-wip-us.apache.org/repos/asf/spark.git + pushd spark + git checkout --force $GIT_TAG + + # Substitute in case published version is different than released + old="^\( \{2,4\}\) ${RELEASE_VERSION}<\/version>$" + new="\1 ${PUBLISH_VERSION}<\/version>" + find . -name pom.xml | grep -v dev | xargs -I {} sed -i \ + -e "s/${old}/${new}/" {} # Using Nexus API documented here: # https://support.sonatype.com/entries/39720203-Uploading-to-a-Staging-Repository-via-REST-API echo "Creating Nexus staging repository" - repo_request=" " + repo_request="Apache Spark $GIT_TAG " out=$(curl -X POST -d "$repo_request" -u $ASF_USERNAME:$ASF_PASSWORD \ -H "Content-Type:application/xml" -v \ $NEXUS_ROOT/profiles/$NEXUS_PROFILE/start) @@ -101,13 +118,13 @@ if [[ ! "$@" =~ --package-only ]]; then rm -rf $SPARK_REPO - mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + build/mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ clean install ./dev/change-version-to-2.11.sh - - mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + + build/mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dscala-2.11 -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ clean install @@ -144,7 +161,7 @@ if [[ ! "$@" =~ --package-only ]]; then done echo "Closing nexus staging repository" - repo_request="Apache Spark $GIT_TAG (published as $PUBLISH_VERSION) " + repo_request="$staged_repo_id Apache Spark $GIT_TAG " out=$(curl -X POST -d "$repo_request" -u $ASF_USERNAME:$ASF_PASSWORD \ -H "Content-Type:application/xml" -v \ $NEXUS_ROOT/profiles/$NEXUS_PROFILE/finish) @@ -155,88 +172,96 @@ if [[ ! "$@" =~ --package-only ]]; then rm -rf spark fi -# Source and binary tarballs -echo "Packaging release tarballs" -git clone https://git-wip-us.apache.org/repos/asf/spark.git -cd spark -git checkout --force $GIT_TAG -release_hash=`git rev-parse HEAD` - -rm .gitignore -rm -rf .git -cd .. - -cp -r spark spark-$RELEASE_VERSION -tar cvzf spark-$RELEASE_VERSION.tgz spark-$RELEASE_VERSION -echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour --output spark-$RELEASE_VERSION.tgz.asc \ - --detach-sig spark-$RELEASE_VERSION.tgz -echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md MD5 spark-$RELEASE_VERSION.tgz > \ - spark-$RELEASE_VERSION.tgz.md5 -echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md SHA512 spark-$RELEASE_VERSION.tgz > \ - spark-$RELEASE_VERSION.tgz.sha -rm -rf spark-$RELEASE_VERSION - -make_binary_release() { - NAME=$1 - FLAGS=$2 - cp -r spark spark-$RELEASE_VERSION-bin-$NAME - - cd spark-$RELEASE_VERSION-bin-$NAME - - # TODO There should probably be a flag to make-distribution to allow 2.11 support - if [[ $FLAGS == *scala-2.11* ]]; then - ./dev/change-version-to-2.11.sh - fi - - ./make-distribution.sh --name $NAME --tgz $FLAGS 2>&1 | tee ../binary-release-$NAME.log +if [[ ! "$@" =~ --skip-package ]]; then + # Source and binary tarballs + echo "Packaging release tarballs" + git clone https://git-wip-us.apache.org/repos/asf/spark.git + cd spark + git checkout --force $GIT_TAG + release_hash=`git rev-parse HEAD` + + rm .gitignore + rm -rf .git cd .. - cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . - rm -rf spark-$RELEASE_VERSION-bin-$NAME - - echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \ - --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \ - --detach-sig spark-$RELEASE_VERSION-bin-$NAME.tgz - echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ - MD5 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ - spark-$RELEASE_VERSION-bin-$NAME.tgz.md5 - echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ - SHA512 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ - spark-$RELEASE_VERSION-bin-$NAME.tgz.sha -} - - -make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" & -make_binary_release "hadoop1-scala2.11" "-Phive -Dscala-2.11" & -make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & -make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" & -make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" & -make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" & -make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" & -make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" & -wait - -# Copy data -echo "Copying release tarballs" -rc_folder=spark-$RELEASE_VERSION-$RC_NAME -ssh $ASF_USERNAME@people.apache.org \ - mkdir /home/$ASF_USERNAME/public_html/$rc_folder -scp spark-* \ - $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_folder/ - -# Docs -cd spark -build/sbt clean -cd docs -# Compile docs with Java 7 to use nicer format -JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build -echo "Copying release documentation" -rc_docs_folder=${rc_folder}-docs -ssh $ASF_USERNAME@people.apache.org \ - mkdir /home/$ASF_USERNAME/public_html/$rc_docs_folder -rsync -r _site/* $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_docs_folder - -echo "Release $RELEASE_VERSION completed:" -echo "Git tag:\t $GIT_TAG" -echo "Release commit:\t $release_hash" -echo "Binary location:\t http://people.apache.org/~$ASF_USERNAME/$rc_folder" -echo "Doc location:\t http://people.apache.org/~$ASF_USERNAME/$rc_docs_folder" + + cp -r spark spark-$RELEASE_VERSION + tar cvzf spark-$RELEASE_VERSION.tgz spark-$RELEASE_VERSION + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour --output spark-$RELEASE_VERSION.tgz.asc \ + --detach-sig spark-$RELEASE_VERSION.tgz + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md MD5 spark-$RELEASE_VERSION.tgz > \ + spark-$RELEASE_VERSION.tgz.md5 + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md SHA512 spark-$RELEASE_VERSION.tgz > \ + spark-$RELEASE_VERSION.tgz.sha + rm -rf spark-$RELEASE_VERSION + + # Updated for each binary build + make_binary_release() { + NAME=$1 + FLAGS=$2 + ZINC_PORT=$3 + cp -r spark spark-$RELEASE_VERSION-bin-$NAME + + cd spark-$RELEASE_VERSION-bin-$NAME + + # TODO There should probably be a flag to make-distribution to allow 2.11 support + if [[ $FLAGS == *scala-2.11* ]]; then + ./dev/change-version-to-2.11.sh + fi + + export ZINC_PORT=$ZINC_PORT + echo "Creating distribution: $NAME ($FLAGS)" + ./make-distribution.sh --name $NAME --tgz $FLAGS -DzincPort=$ZINC_PORT 2>&1 > \ + ../binary-release-$NAME.log + cd .. + cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . + + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \ + --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \ + --detach-sig spark-$RELEASE_VERSION-bin-$NAME.tgz + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ + MD5 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ + spark-$RELEASE_VERSION-bin-$NAME.tgz.md5 + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ + SHA512 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ + spark-$RELEASE_VERSION-bin-$NAME.tgz.sha + } + + # We increment the Zinc port each time to avoid OOM's and other craziness if multiple builds + # share the same Zinc server. + make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" "3030" & + make_binary_release "hadoop1-scala2.11" "-Phive -Dscala-2.11" "3031" & + make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" "3032" & + make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" "3033" & + make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" "3034" & + make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" "3035" & + make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" "3036" & + make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" "3037" & + wait + rm -rf spark-$RELEASE_VERSION-bin-*/ + + # Copy data + echo "Copying release tarballs" + rc_folder=spark-$RELEASE_VERSION-$RC_NAME + ssh $ASF_USERNAME@people.apache.org \ + mkdir /home/$ASF_USERNAME/public_html/$rc_folder + scp spark-* \ + $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_folder/ + + # Docs + cd spark + sbt/sbt clean + cd docs + # Compile docs with Java 7 to use nicer format + JAVA_HOME="$JAVA_7_HOME" PRODUCTION=1 RELEASE_VERSION="$RELEASE_VERSION" jekyll build + echo "Copying release documentation" + rc_docs_folder=${rc_folder}-docs + ssh $ASF_USERNAME@people.apache.org \ + mkdir /home/$ASF_USERNAME/public_html/$rc_docs_folder + rsync -r _site/* $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_docs_folder + + echo "Release $RELEASE_VERSION completed:" + echo "Git tag:\t $GIT_TAG" + echo "Release commit:\t $release_hash" + echo "Binary location:\t http://people.apache.org/~$ASF_USERNAME/$rc_folder" + echo "Doc location:\t http://people.apache.org/~$ASF_USERNAME/$rc_docs_folder" +fi diff --git a/dev/create-release/known_translations b/dev/create-release/known_translations index b74e4ee8a330b..0a599b5a65549 100644 --- a/dev/create-release/known_translations +++ b/dev/create-release/known_translations @@ -57,3 +57,37 @@ watermen - Yadong Qi witgo - Guoqiang Li xinyunh - Xinyun Huang zsxwing - Shixiong Zhu +Bilna - Bilna P +DoingDone9 - Doing Done +Earne - Ernest +FlytxtRnD - Meethu Mathew +GenTang - Gen TANG +JoshRosen - Josh Rosen +MechCoder - Manoj Kumar +OopsOutOfMemory - Sheng Li +Peishen-Jia - Peishen Jia +SaintBacchus - Huang Zhaowei +azagrebin - Andrey Zagrebin +bzz - Alexander Bezzubov +fjiang6 - Fan Jiang +gasparms - Gaspar Munoz +guowei2 - Guo Wei +hhbyyh - Yuhao Yang +hseagle - Peng Xu +javadba - Stephen Boesch +jbencook - Ben Cook +kul - Kuldeep +ligangty - Gang Li +marsishandsome - Liangliang Gu +medale - Markus Dale +nemccarthy - Nathan McCarthy +nxwhite-str - Nate Crosswhite +seayi - Xiaohua Yi +tianyi - Yi Tian +uncleGen - Uncle Gen +viper-kun - Xu Kun +x1- - Yuri Saito +zapletal-martin - Martin Zapletal +zuxqoj - Shekhar Bansal +mingyukim - Mingyu Kim +sigmoidanalytics - Mayur Rustagi diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index dfa924d2aa0ba..3062e9c3c6651 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -244,6 +244,8 @@ def resolve_jira_issue(merge_branches, comment, default_jira_id=""): versions = asf_jira.project_versions("SPARK") versions = sorted(versions, key=lambda x: x.name, reverse=True) versions = filter(lambda x: x.raw['released'] is False, versions) + # Consider only x.y.z versions + versions = filter(lambda x: re.match('\d+\.\d+\.\d+', x.name), versions) default_fix_versions = map(lambda x: fix_version_from_branch(x, versions).name, merge_branches) for v in default_fix_versions: diff --git a/dev/run-tests b/dev/run-tests index 2257a566bb1bb..d6935a61c6d29 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -36,7 +36,7 @@ function handle_error () { } -# Build against the right verison of Hadoop. +# Build against the right version of Hadoop. { if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then if [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop1.0" ]; then @@ -77,7 +77,7 @@ export SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Pkinesis-asl" fi } -# Only run Hive tests if there are sql changes. +# Only run Hive tests if there are SQL changes. # Partial solution for SPARK-1455. if [ -n "$AMPLAB_JENKINS" ]; then git fetch origin master:master @@ -141,29 +141,41 @@ echo "=========================================================================" CURRENT_BLOCK=$BLOCK_BUILD { + HIVE_BUILD_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver" + HIVE_12_BUILD_ARGS="$HIVE_BUILD_ARGS -Phive-0.12.0" - # NOTE: echo "q" is needed because sbt on encountering a build file with failure - # (either resolution or compilation) prompts the user for input either q, r, etc - # to quit or retry. This echo is there to make it not block. - # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a - # single argument! - # QUESTION: Why doesn't 'yes "q"' work? - # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? # First build with Hive 0.12.0 to ensure patches do not break the Hive 0.12.0 build - HIVE_12_BUILD_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver -Phive-0.12.0" echo "[info] Compile with Hive 0.12.0" - echo -e "q\n" \ - | build/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ - | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + [ -d "lib_managed" ] && rm -rf lib_managed + echo "[info] Building Spark with these arguments: $HIVE_12_BUILD_ARGS" + + if [ "${AMPLAB_JENKINS_BUILD_TOOL}" == "maven" ]; then + build/mvn $HIVE_12_BUILD_ARGS clean package -DskipTests + else + # NOTE: echo "q" is needed because sbt on encountering a build file with failure + # (either resolution or compilation) prompts the user for input either q, r, etc + # to quit or retry. This echo is there to make it not block. + # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a + # single argument! + # QUESTION: Why doesn't 'yes "q"' work? + # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? + echo -e "q\n" \ + | build/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ + | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + fi # Then build with default Hive version (0.13.1) because tests are based on this version echo "[info] Compile with Hive 0.13.1" - rm -rf lib_managed - echo "[info] Building Spark with these arguments: $SBT_MAVEN_PROFILES_ARGS"\ - " -Phive -Phive-thriftserver" - echo -e "q\n" \ - | build/sbt $SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver package assembly/assembly \ - | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + [ -d "lib_managed" ] && rm -rf lib_managed + echo "[info] Building Spark with these arguments: $HIVE_BUILD_ARGS" + + if [ "${AMPLAB_JENKINS_BUILD_TOOL}" == "maven" ]; then + build/mvn $HIVE_BUILD_ARGS clean package -DskipTests + else + echo -e "q\n" \ + | build/sbt $HIVE_BUILD_ARGS package assembly/assembly \ + | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + fi } echo "" @@ -183,24 +195,28 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS if [ -n "$_SQL_TESTS_ONLY" ]; then # This must be an array of individual arguments. Otherwise, having one long string # will be interpreted as a single test, which doesn't work. - SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "mllib/test") + SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "hive-thriftserver/test" "mllib/test") else SBT_MAVEN_TEST_ARGS=("test") fi echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS ${SBT_MAVEN_TEST_ARGS[@]}" - # NOTE: echo "q" is needed because sbt on encountering a build file with failure - # (either resolution or compilation) prompts the user for input either q, r, etc - # to quit or retry. This echo is there to make it not block. - # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a - # single argument! - # "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. - # QUESTION: Why doesn't 'yes "q"' work? - # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? - echo -e "q\n" \ - | build/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \ - | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + if [ "${AMPLAB_JENKINS_BUILD_TOOL}" == "maven" ]; then + build/mvn test $SBT_MAVEN_PROFILES_ARGS --fail-at-end + else + # NOTE: echo "q" is needed because sbt on encountering a build file with failure + # (either resolution or compilation) prompts the user for input either q, r, etc + # to quit or retry. This echo is there to make it not block. + # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a + # single argument! + # "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. + # QUESTION: Why doesn't 'yes "q"' work? + # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? + echo -e "q\n" \ + | build/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \ + | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + fi } echo "" diff --git a/docs/_config.yml b/docs/_config.yml index e2db274e1f619..0652927a8ce9b 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -10,6 +10,7 @@ kramdown: include: - _static + - _modules # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 8841f7675d35e..2e88b3093652d 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -7,7 +7,9 @@$staged_repo_id Apache Spark $GIT_TAG (published as $PUBLISH_VERSION) {{ page.title }} - Spark {{site.SPARK_VERSION_SHORT}} Documentation - + {% if page.description %} + + {% endif %} {% if page.redirect %} @@ -69,7 +71,7 @@- Spark Programming Guide
- Spark Streaming
-- Spark SQL
+- DataFrames and SQL
- MLlib (Machine Learning)
- GraphX (Graph Processing)
- Bagel (Pregel on Spark)
diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index 7e55131754a3f..c2fe6b0e286ce 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -1,6 +1,7 @@ --- layout: global -title: Bagel Programming Guide +displayTitle: Bagel Programming Guide +title: Bagel --- **Bagel will soon be superseded by [GraphX](graphx-programming-guide.html); we recommend that new users try GraphX instead.** diff --git a/docs/building-spark.md b/docs/building-spark.md index fb93017861ed0..57d0ca834f460 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -9,6 +9,10 @@ redirect_from: "building-with-maven.html" Building Spark using Maven requires Maven 3.0.4 or newer and Java 6+. +**Note:** Building Spark with Java 7 or later can create JAR files that may not be +readable with early versions of Java 6, due to the large number of files in the JAR +archive. Build with Java 6 if this is an issue for your deployment. + # Building with `build/mvn` Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: @@ -111,9 +115,9 @@ To produce a Spark package compiled with Scala 2.11, use the `-Dscala-2.11` prop dev/change-version-to-2.11.sh mvn -Pyarn -Phadoop-2.4 -Dscala-2.11 -DskipTests clean package -Scala 2.11 support in Spark is experimental and does not support a few features. -Specifically, Spark's external Kafka library and JDBC component are not yet -supported in Scala 2.11 builds. +Scala 2.11 support in Spark does not support a few features due to dependencies +which are themselves not Scala 2.11 ready. Specifically, Spark's external +Kafka library and JDBC component are not yet supported in Scala 2.11 builds. # Spark Tests in Maven @@ -137,15 +141,18 @@ We use the scala-maven-plugin which supports incremental and continuous compilat should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. A couple of gotchas to note: + * it only scans the paths `src/main` and `src/test` (see [docs](http://scala-tools.org/mvnsites/maven-scala-plugin/usage_cc.html)), so it will only work from within certain submodules that have that structure. + * you'll typically need to run `mvn install` from the project root for compilation within specific submodules to work; this is because submodules that depend on other submodules do so via the `spark-parent` module). Thus, the full flow for running continuous-compilation of the `core` submodule may look more like: - ``` + +``` $ mvn install $ cd core $ mvn scala:cc @@ -156,14 +163,6 @@ Thus, the full flow for running continuous-compilation of the `core` submodule m For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troubleshooting, refer to the [wiki page for IDE setup](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-IDESetup). -# Building Spark Debian Packages - -The Maven build includes support for building a Debian package containing the assembly 'fat-jar', PySpark, and the necessary scripts and configuration files. This can be created by specifying the following: - - mvn -Pdeb -DskipTests clean package - -The debian package can then be found under assembly/target. We added the short commit hash to the file name so that we can distinguish individual packages built for SNAPSHOT versions. - # Running Java 8 Test Suites Running only Java 8 tests and nothing else. diff --git a/docs/configuration.md b/docs/configuration.md index e4e4b8d516b75..63fc99e7d3e29 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1,6 +1,7 @@ --- layout: global -title: Spark Configuration +displayTitle: Spark Configuration +title: Configuration --- * This will become a table of contents (this text will be scraped). {:toc} @@ -69,7 +70,9 @@ each line consists of a key and a value separated by whitespace. For example: Any values specified as flags or in the properties file will be passed on to the application and merged with those specified through SparkConf. Properties set directly on the SparkConf take highest precedence, then flags passed to `spark-submit` or `spark-shell`, then options -in the `spark-defaults.conf` file. +in the `spark-defaults.conf` file. A few configuration keys have been renamed since earlier +versions of Spark; in such cases, the older key names are still accepted, but take lower +precedence than any instance of the newer key. ## Viewing Spark Properties @@ -93,14 +96,6 @@ of the most common options to set are: The name of your application. This will appear in the UI and in log data. -- -spark.master
(none) -- The cluster manager to connect to. See the list of - allowed master URL's. - -spark.driver.cores
1 @@ -108,23 +103,6 @@ of the most common options to set are: Number of cores to use for the driver process, only in cluster mode.- - -spark.driver.memory
512m -- Amount of memory to use for the driver process, i.e. where SparkContext is initialized. - (e.g. -512m
,2g
). -- - -spark.executor.memory
512m -- Amount of memory to use per executor process, in the same format as JVM memory strings - (e.g. -512m
,2g
). -spark.driver.maxResultSize
1g @@ -137,38 +115,35 @@ of the most common options to set are: - -spark.serializer
org.apache.spark.serializer. +
JavaSerializer +spark.driver.memory
512m - Class to use for serializing objects that will be sent over the network or need to be cached - in serialized form. The default of Java serialization works with any Serializable Java object - but is quite slow, so we recommend using - org.apache.spark.serializer.KryoSerializer
and configuring Kryo serialization - when speed is necessary. Can be any subclass of - -org.apache.spark.Serializer
. + Amount of memory to use for the driver process, i.e. where SparkContext is initialized. + (e.g.512m
,2g
). + +
Note: In client mode, this config must not be set through theSparkConf
+ directly in your application, because the driver JVM has already started at that point. + Instead, please set this through the--driver-memory
command line option + or in your default properties file.- -spark.kryo.classesToRegister
(none) + +spark.executor.memory
512m - If you use Kryo serialization, give a comma-separated list of custom class names to register - with Kryo. - See the tuning guide for more details. + Amount of memory to use per executor process, in the same format as JVM memory strings + (e.g. 512m
,2g
).- +spark.kryo.registrator
spark.extraListeners
(none) - If you use Kryo serialization, set this class to register your custom classes with Kryo. This - property is useful if you need to register your classes in a custom way, e.g. to specify a custom - field serializer. Otherwise spark.kryo.classesToRegister
is simpler. It should be - set to a class that extends - -KryoRegistrator
. - See the tuning guide for more details. + A comma-separated list of classes that implementSparkListener
; when initializing + SparkContext, instances of these classes will be created and registered with Spark's listener + bus. If a class has a single-argument constructor that accepts a SparkConf, that constructor + will be called; otherwise, a zero-argument constructor will be called. If no valid constructor + can be found, the SparkContext creation will fail with an exception.@@ -190,6 +165,14 @@ of the most common options to set are: Logs the effective SparkConf as INFO when a SparkContext is started. ++ +spark.master
(none) ++ The cluster manager to connect to. See the list of + allowed master URL's. + +
Logs | else Seq.empty} {if (threadDumpEnabled)Thread Dump | else Seq.empty} - {execInfoSorted.map(execRow)} + {execInfoSorted.map(execRow(_, logsExist))}
---|
Executor ID | @@ -44,12 +58,32 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: StageTotal Tasks | Failed Tasks | Succeeded Tasks | -Input | -Output | -Shuffle Read | -Shuffle Write | -Shuffle Spill (Memory) | -Shuffle Spill (Disk) | + {if (hasInput) { ++ Input Size / Records + | + }} + {if (hasOutput) { ++ Output Size / Records + | + }} + {if (hasShuffleRead) { ++ + Shuffle Read Size / Records + | + }} + {if (hasShuffleWrite) { ++ + Shuffle Write Size / Records + | + }} + {if (hasBytesSpilled) { +Shuffle Spill (Memory) | +Shuffle Spill (Disk) | + }} {createExecutorTable()} @@ -76,18 +110,34 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage{v.failedTasks + v.succeededTasks} | {v.failedTasks} | {v.succeededTasks} | -- {Utils.bytesToString(v.inputBytes)} | -- {Utils.bytesToString(v.outputBytes)} | -- {Utils.bytesToString(v.shuffleRead)} | -- {Utils.bytesToString(v.shuffleWrite)} | -- {Utils.bytesToString(v.memoryBytesSpilled)} | -- {Utils.bytesToString(v.diskBytesSpilled)} | + {if (stageData.hasInput) { ++ {s"${Utils.bytesToString(v.inputBytes)} / ${v.inputRecords}"} + | + }} + {if (stageData.hasOutput) { ++ {s"${Utils.bytesToString(v.outputBytes)} / ${v.outputRecords}"} + | + }} + {if (stageData.hasShuffleRead) { ++ {s"${Utils.bytesToString(v.shuffleRead)} / ${v.shuffleReadRecords}"} + | + }} + {if (stageData.hasShuffleWrite) { ++ {s"${Utils.bytesToString(v.shuffleWrite)} / ${v.shuffleWriteRecords}"} + | + }} + {if (stageData.hasBytesSpilled) { ++ {Utils.bytesToString(v.memoryBytesSpilled)} + | ++ {Utils.bytesToString(v.diskBytesSpilled)} + | + }} } case None => diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 77d36209c6048..7541d3e9c72e7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -32,7 +32,10 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { - val jobId = request.getParameter("id").toInt + val parameterId = request.getParameter("id") + require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") + + val jobId = parameterId.toInt val jobDataOption = listener.jobIdToData.get(jobId) if (jobDataOption.isEmpty) { val content = 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 4d200eeda86b9..937d95a934b59 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 @@ -203,6 +203,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { for (stageId <- jobData.stageIds) { stageIdToActiveJobIds.get(stageId).foreach { jobsUsingStage => jobsUsingStage.remove(jobEnd.jobId) + if (jobsUsingStage.isEmpty) { + stageIdToActiveJobIds.remove(stageId) + } stageIdToInfo.get(stageId).foreach { stageInfo => if (stageInfo.submissionTime.isEmpty) { // if this stage is pending, it won't complete, so mark it as "skipped": @@ -394,24 +397,48 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageData.shuffleWriteBytes += shuffleWriteDelta execSummary.shuffleWrite += shuffleWriteDelta + val shuffleWriteRecordsDelta = + (taskMetrics.shuffleWriteMetrics.map(_.shuffleRecordsWritten).getOrElse(0L) + - oldMetrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleRecordsWritten).getOrElse(0L)) + stageData.shuffleWriteRecords += shuffleWriteRecordsDelta + execSummary.shuffleWriteRecords += shuffleWriteRecordsDelta + val shuffleReadDelta = - (taskMetrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L) - - oldMetrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead).getOrElse(0L)) - stageData.shuffleReadBytes += shuffleReadDelta + (taskMetrics.shuffleReadMetrics.map(_.totalBytesRead).getOrElse(0L) + - oldMetrics.flatMap(_.shuffleReadMetrics).map(_.totalBytesRead).getOrElse(0L)) + stageData.shuffleReadTotalBytes += shuffleReadDelta execSummary.shuffleRead += shuffleReadDelta + val shuffleReadRecordsDelta = + (taskMetrics.shuffleReadMetrics.map(_.recordsRead).getOrElse(0L) + - oldMetrics.flatMap(_.shuffleReadMetrics).map(_.recordsRead).getOrElse(0L)) + stageData.shuffleReadRecords += shuffleReadRecordsDelta + execSummary.shuffleReadRecords += shuffleReadRecordsDelta + val inputBytesDelta = (taskMetrics.inputMetrics.map(_.bytesRead).getOrElse(0L) - oldMetrics.flatMap(_.inputMetrics).map(_.bytesRead).getOrElse(0L)) stageData.inputBytes += inputBytesDelta execSummary.inputBytes += inputBytesDelta + val inputRecordsDelta = + (taskMetrics.inputMetrics.map(_.recordsRead).getOrElse(0L) + - oldMetrics.flatMap(_.inputMetrics).map(_.recordsRead).getOrElse(0L)) + stageData.inputRecords += inputRecordsDelta + execSummary.inputRecords += inputRecordsDelta + val outputBytesDelta = (taskMetrics.outputMetrics.map(_.bytesWritten).getOrElse(0L) - oldMetrics.flatMap(_.outputMetrics).map(_.bytesWritten).getOrElse(0L)) stageData.outputBytes += outputBytesDelta execSummary.outputBytes += outputBytesDelta + val outputRecordsDelta = + (taskMetrics.outputMetrics.map(_.recordsWritten).getOrElse(0L) + - oldMetrics.flatMap(_.outputMetrics).map(_.recordsWritten).getOrElse(0L)) + stageData.outputRecords += outputRecordsDelta + execSummary.outputRecords += outputRecordsDelta + val diskSpillDelta = taskMetrics.diskBytesSpilled - oldMetrics.map(_.diskBytesSpilled).getOrElse(0L) stageData.diskBytesSpilled += diskSpillDelta diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 5fc6cc7533150..f47cdc935e539 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -32,6 +32,8 @@ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val poolName = request.getParameter("poolname") + require(poolName != null && poolName.nonEmpty, "Missing poolname parameter") + val poolToActiveStages = listener.poolToActiveStages val activeStages = poolToActiveStages.get(poolName) match { case Some(s) => s.values.toSeq 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 d8be1b20b3acd..110f8780a9a12 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 @@ -36,8 +36,14 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { - val stageId = request.getParameter("id").toInt - val stageAttemptId = request.getParameter("attempt").toInt + val parameterId = request.getParameter("id") + require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") + + val parameterAttempt = request.getParameter("attempt") + require(parameterAttempt != null && parameterAttempt.nonEmpty, "Missing attempt parameter") + + val stageId = parameterId.toInt + val stageAttemptId = parameterAttempt.toInt val stageDataOption = listener.stageIdToData.get((stageId, stageAttemptId)) if (stageDataOption.isEmpty || stageDataOption.get.taskData.isEmpty) { @@ -56,11 +62,6 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val numCompleted = tasks.count(_.taskInfo.finished) val accumulables = listener.stageIdToData((stageId, stageAttemptId)).accumulables val hasAccumulators = accumulables.size > 0 - val hasInput = stageData.inputBytes > 0 - val hasOutput = stageData.outputBytes > 0 - val hasShuffleRead = stageData.shuffleReadBytes > 0 - val hasShuffleWrite = stageData.shuffleWriteBytes > 0 - val hasBytesSpilled = stageData.memoryBytesSpilled > 0 && stageData.diskBytesSpilled > 0 val summary =
---|
Property Name | Default | Meaning | ||
---|---|---|---|---|
spark.driver.extraJavaOptions |
+ spark.driver.extraClassPath |
(none) |
- A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.
+ Extra classpath entries to append to the classpath of the driver.
+
+ Note: In client mode, this config must not be set through the SparkConf
+ directly in your application, because the driver JVM has already started at that point.
+ Instead, please set this through the --driver-class-path command line option or in
+ your default properties file. |
|
spark.driver.extraClassPath |
+ spark.driver.extraJavaOptions |
(none) |
- Extra classpath entries to append to the classpath of the driver.
+ A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.
+
+ Note: In client mode, this config must not be set through the SparkConf
+ directly in your application, because the driver JVM has already started at that point.
+ Instead, please set this through the --driver-java-options command line option or in
+ your default properties file. |
|
(none) |
Set a special library path to use when launching the driver JVM.
+
+ Note: In client mode, this config must not be set through the SparkConf
+ directly in your application, because the driver JVM has already started at that point.
+ Instead, please set this through the --driver-library-path command line option or in
+ your default properties file. |
|||
spark.executor.extraJavaOptions |
- (none) | +spark.driver.userClassPathFirst |
+ false | - A string of extra JVM options to pass to executors. For instance, GC settings or other - logging. Note that it is illegal to set Spark properties or heap size settings with this - option. Spark properties should be set using a SparkConf object or the - spark-defaults.conf file used with the spark-submit script. Heap size settings can be set - with spark.executor.memory. + (Experimental) Whether to give user-added jars precedence over Spark's own jars when loading + classes in the the driver. This feature can be used to mitigate conflicts between Spark's + dependencies and user dependencies. It is currently an experimental feature. + + This is used in cluster mode only. |
spark.executor.extraClassPath |
(none) | - Extra classpath entries to append to the classpath of executors. This exists primarily - for backwards-compatibility with older versions of Spark. Users typically should not need - to set this option. + Extra classpath entries to append to the classpath of executors. This exists primarily for + backwards-compatibility with older versions of Spark. Users typically should not need to set + this option. | ||
spark.executor.extraLibraryPath |
+ spark.executor.extraJavaOptions |
(none) | - Set a special library path to use when launching executor JVM's. + A string of extra JVM options to pass to executors. For instance, GC settings or other logging. + Note that it is illegal to set Spark properties or heap size settings with this option. Spark + properties should be set using a SparkConf object or the spark-defaults.conf file used with the + spark-submit script. Heap size settings can be set with spark.executor.memory. | |
spark.executor.logs.rolling.strategy |
+ spark.executor.extraLibraryPath |
(none) |
- Set the strategy of rolling of executor logs. By default it is disabled. It can
- be set to "time" (time-based rolling) or "size" (size-based rolling). For "time",
- use spark.executor.logs.rolling.time.interval to set the rolling interval.
- For "size", use spark.executor.logs.rolling.size.maxBytes to set
- the maximum file size for rolling.
+ Set a special library path to use when launching executor JVM's.
|
|
spark.executor.logs.rolling.time.interval |
- daily | +spark.executor.logs.rolling.maxRetainedFiles |
+ (none) |
- Set the time interval by which the executor logs will be rolled over.
- Rolling is disabled by default. Valid values are `daily`, `hourly`, `minutely` or
- any interval in seconds. See spark.executor.logs.rolling.maxRetainedFiles
- for automatic cleaning of old logs.
+ Sets the number of latest rolling log files that are going to be retained by the system.
+ Older log files will be deleted. Disabled by default.
|
spark.executor.logs.rolling.maxRetainedFiles |
+ spark.executor.logs.rolling.strategy |
(none) |
- Sets the number of latest rolling log files that are going to be retained by the system.
- Older log files will be deleted. Disabled by default.
+ Set the strategy of rolling of executor logs. By default it is disabled. It can
+ be set to "time" (time-based rolling) or "size" (size-based rolling). For "time",
+ use spark.executor.logs.rolling.time.interval to set the rolling interval.
+ For "size", use spark.executor.logs.rolling.size.maxBytes to set
+ the maximum file size for rolling.
+ |
+|
spark.executor.logs.rolling.time.interval |
+ daily | +
+ Set the time interval by which the executor logs will be rolled over.
+ Rolling is disabled by default. Valid values are `daily`, `hourly`, `minutely` or
+ any interval in seconds. See spark.executor.logs.rolling.maxRetainedFiles
+ for automatic cleaning of old logs.
|
||
spark.files.userClassPathFirst |
+ spark.executor.userClassPathFirst |
false |
- (Experimental) Whether to give user-added jars precedence over Spark's own jars when
- loading classes in Executors. This feature can be used to mitigate conflicts between
- Spark's dependencies and user dependencies. It is currently an experimental feature.
- (Currently, this setting does not work for YARN, see SPARK-2996 for more details).
+ (Experimental) Same functionality as spark.driver.userClassPathFirst , but
+ applied to executor instances.
|
|
spark.python.worker.memory |
- 512m | +spark.executorEnv.[EnvironmentVariableName] |
+ (none) |
- Amount of memory to use per python worker process during aggregation, in the same
- format as JVM memory strings (e.g. 512m , 2g ). If the memory
- used during aggregation goes above this amount, it will spill the data into disks.
+ Add the environment variable specified by EnvironmentVariableName to the Executor
+ process. The user can specify multiple of these to set multiple environment variables.
|
spark.python.worker.memory |
+ 512m | +
+ Amount of memory to use per python worker process during aggregation, in the same
+ format as JVM memory strings (e.g. 512m , 2g ). If the memory
+ used during aggregation goes above this amount, it will spill the data into disks.
+ |
+||
spark.python.worker.reuse |
true | @@ -336,40 +350,38 @@ Apart from these, the following properties are also available, and may be useful from JVM to Python worker for every task.
Property Name | Default | Meaning | ||
---|---|---|---|---|
spark.executorEnv.[EnvironmentVariableName] |
- (none) | +spark.reducer.maxMbInFlight |
+ 48 |
- Add the environment variable specified by EnvironmentVariableName to the Executor
- process. The user can specify multiple of these to set multiple environment variables.
+ Maximum size (in megabytes) of map outputs to fetch simultaneously from each reduce task. Since
+ each output requires us to create a buffer to receive it, this represents a fixed memory
+ overhead per reduce task, so keep it small unless you have a large amount of memory.
|
spark.mesos.executor.home |
- driver side SPARK_HOME |
+ spark.shuffle.blockTransferService |
+ netty |
- Set the directory in which Spark is installed on the executors in Mesos. By default, the
- executors will simply use the driver's Spark home directory, which may not be visible to
- them. Note that this is only relevant if a Spark binary package is not specified through
- spark.executor.uri .
+ Implementation to use for transferring shuffle and cached blocks between executors. There
+ are two implementations available: netty and nio . Netty-based
+ block transfer is intended to be simpler but equally efficient and is the default option
+ starting in 1.2.
|
spark.mesos.executor.memoryOverhead |
- executor memory * 0.07, with minimum of 384 | +spark.shuffle.compress |
+ true |
- This value is an additive for spark.executor.memory , specified in MiB,
- which is used to calculate the total Mesos task memory. A value of 384
- implies a 384MiB overhead. Additionally, there is a hard-coded 7% minimum
- overhead. The final overhead will be the larger of either
- `spark.mesos.executor.memoryOverhead` or 7% of `spark.executor.memory`.
+ Whether to compress map output files. Generally a good idea. Compression will use
+ spark.io.compression.codec .
|
Property Name | Default | Meaning | ||
---|---|---|---|---|
spark.shuffle.consolidateFiles |
false | @@ -381,55 +393,46 @@ Apart from these, the following properties are also available, and may be useful|||
spark.shuffle.spill |
- true | +spark.shuffle.file.buffer.kb |
+ 32 |
- If set to "true", limits the amount of memory used during reduces by spilling data out to disk.
- This spilling threshold is specified by spark.shuffle.memoryFraction .
+ Size of the in-memory buffer for each shuffle file output stream, in kilobytes. These buffers
+ reduce the number of disk seeks and system calls made in creating intermediate shuffle files.
|
spark.shuffle.spill.compress |
- true | +spark.shuffle.io.maxRetries |
+ 3 |
- Whether to compress data spilled during shuffles. Compression will use
- spark.io.compression.codec .
+ (Netty only) Fetches that fail due to IO-related exceptions are automatically retried if this is
+ set to a non-zero value. This retry logic helps stabilize large shuffles in the face of long GC
+ pauses or transient network connectivity issues.
|
spark.shuffle.memoryFraction |
- 0.2 | +spark.shuffle.io.numConnectionsPerPeer |
+ 1 |
- Fraction of Java heap to use for aggregation and cogroups during shuffles, if
- spark.shuffle.spill is true. At any given time, the collective size of
- all in-memory maps used for shuffles is bounded by this limit, beyond which the contents will
- begin to spill to disk. If spills are often, consider increasing this value at the expense of
- spark.storage.memoryFraction .
+ (Netty only) Connections between hosts are reused in order to reduce connection buildup for
+ large clusters. For clusters with many hard disks and few hosts, this may result in insufficient
+ concurrency to saturate all disks, and so users may consider increasing this value.
|
spark.shuffle.compress |
+ spark.shuffle.io.preferDirectBufs |
true |
- Whether to compress map output files. Generally a good idea. Compression will use
- spark.io.compression.codec .
- |
-|
spark.shuffle.file.buffer.kb |
- 32 | -- Size of the in-memory buffer for each shuffle file output stream, in kilobytes. These buffers - reduce the number of disk seeks and system calls made in creating intermediate shuffle files. + (Netty only) Off-heap buffers are used to reduce garbage collection during shuffle and cache + block transfer. For environments where off-heap memory is tightly limited, users may wish to + turn this off to force all allocations from Netty to be on-heap. | ||
spark.reducer.maxMbInFlight |
- 48 | +spark.shuffle.io.retryWait |
+ 5 |
- Maximum size (in megabytes) of map outputs to fetch simultaneously from each reduce task. Since
- each output requires us to create a buffer to receive it, this represents a fixed memory
- overhead per reduce task, so keep it small unless you have a large amount of memory.
+ (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying
+ is simply maxRetries * retryWait , by default 15 seconds.
|
spark.shuffle.memoryFraction |
+ 0.2 | +
+ Fraction of Java heap to use for aggregation and cogroups during shuffles, if
+ spark.shuffle.spill is true. At any given time, the collective size of
+ all in-memory maps used for shuffles is bounded by this limit, beyond which the contents will
+ begin to spill to disk. If spills are often, consider increasing this value at the expense of
+ spark.storage.memoryFraction .
+ |
+||
spark.shuffle.sort.bypassMergeThreshold |
200 | @@ -450,13 +464,19 @@ Apart from these, the following properties are also available, and may be useful|||
spark.shuffle.blockTransferService |
- netty | +spark.shuffle.spill |
+ true |
- Implementation to use for transferring shuffle and cached blocks between executors. There
- are two implementations available: netty and nio . Netty-based
- block transfer is intended to be simpler but equally efficient and is the default option
- starting in 1.2.
+ If set to "true", limits the amount of memory used during reduces by spilling data out to disk.
+ This spilling threshold is specified by spark.shuffle.memoryFraction .
+ |
+
spark.shuffle.spill.compress |
+ true | +
+ Whether to compress data spilled during shuffles. Compression will use
+ spark.io.compression.codec .
|
Property Name | Default | Meaning | ||
---|---|---|---|---|
spark.ui.port |
- 4040 | +spark.eventLog.compress |
+ false |
- Port for your application's dashboard, which shows memory and workload data.
+ Whether to compress logged events, if spark.eventLog.enabled is true.
|
spark.ui.retainedStages |
- 1000 | +spark.eventLog.dir |
+ file:///tmp/spark-events |
- How many stages the Spark UI and status APIs remember before garbage
- collecting.
+ Base directory in which Spark events are logged, if spark.eventLog.enabled is true.
+ Within this base directory, Spark creates a sub-directory for each application, and logs the
+ events specific to the application in this directory. Users may want to set this to
+ a unified location like an HDFS directory so history files can be read by the history server.
|
spark.ui.retainedJobs |
- 1000 | +spark.eventLog.enabled |
+ false | - How many jobs the Spark UI and status APIs remember before garbage - collecting. + Whether to log Spark events, useful for reconstructing the Web UI after the application has + finished. |
spark.eventLog.enabled |
- false | +spark.ui.port |
+ 4040 | - Whether to log Spark events, useful for reconstructing the Web UI after the application has - finished. + Port for your application's dashboard, which shows memory and workload data. |
spark.eventLog.compress |
- false | +spark.ui.retainedJobs |
+ 1000 |
- Whether to compress logged events, if spark.eventLog.enabled is true.
+ How many jobs the Spark UI and status APIs remember before garbage
+ collecting.
|
spark.eventLog.dir |
- file:///tmp/spark-events | +spark.ui.retainedStages |
+ 1000 |
- Base directory in which Spark events are logged, if spark.eventLog.enabled is true.
- Within this base directory, Spark creates a sub-directory for each application, and logs the
- events specific to the application in this directory. Users may want to set this to
- a unified location like an HDFS directory so history files can be read by the history server.
+ How many stages the Spark UI and status APIs remember before garbage
+ collecting.
|
spark.rdd.compress
spark.closure.serializer
JavaSerializer
StorageLevel.MEMORY_ONLY_SER
). Can save substantial space at the cost of some
- extra CPU time.
+ Serializer class to use for closures. Currently only the Java serializer is supported.
org.apache.spark.io.SnappyCompressionCodec
.
spark.io.compression.snappy.block.size
spark.io.compression.lz4.block.size
spark.closure.serializer
JavaSerializer
spark.io.compression.snappy.block.size
spark.serializer.objectStreamReset
spark.kryo.classesToRegister
spark.kryoserializer.buffer.mb
spark.kryo.registrator
spark.kryoserializer.buffer.max.mb
if needed.
+ If you use Kryo serialization, set this class to register your custom classes with Kryo. This
+ property is useful if you need to register your classes in a custom way, e.g. to specify a custom
+ field serializer. Otherwise spark.kryo.classesToRegister
is simpler. It should be
+ set to a class that extends
+
+ KryoRegistrator
.
+ See the tuning guide for more details.
spark.kryoserializer.buffer.mb
spark.kryoserializer.buffer.max.mb
if needed.
+ spark.rdd.compress
StorageLevel.MEMORY_ONLY_SER
). Can save substantial space at the cost of some
+ extra CPU time.
+ spark.serializer
JavaSerializer
org.apache.spark.serializer.KryoSerializer
and configuring Kryo serialization
+ when speed is necessary. Can be any subclass of
+
+ org.apache.spark.Serializer
.
+ spark.serializer.objectStreamReset
Property Name | Default | Meaning | |||
---|---|---|---|---|---|
spark.broadcast.blockSize |
+ 4096 | +
+ Size of each piece of a block in kilobytes for TorrentBroadcastFactory .
+ Too large a value decreases parallelism during broadcast (makes it slower); however, if it is
+ too small, BlockManager might take a performance hit.
+ |
+|||
spark.broadcast.factory |
+ org.apache.spark.broadcast. TorrentBroadcastFactory |
+ + Which broadcast implementation to use. + | +|||
spark.cleaner.ttl |
+ (infinite) | ++ Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks + generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be + forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in + case of Spark Streaming applications). Note that any RDD that persists in memory for more than + this duration will be cleared as well. + | +|||
spark.default.parallelism |
@@ -649,19 +731,18 @@ Apart from these, the following properties are also available, and may be useful | ||||
spark.broadcast.factory |
- org.apache.spark.broadcast. TorrentBroadcastFactory |
- - Which broadcast implementation to use. - | +spark.executor.heartbeatInterval |
+ 10000 | +Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let + the driver know that the executor is still alive and update it with metrics for in-progress + tasks. |
spark.broadcast.blockSize |
- 4096 | +spark.files.fetchTimeout |
+ 60 |
- Size of each piece of a block in kilobytes for TorrentBroadcastFactory .
- Too large a value decreases parallelism during broadcast (makes it slower); however, if it is
- too small, BlockManager might take a performance hit.
+ Communication timeout to use when fetching files added through SparkContext.addFile() from
+ the driver, in seconds.
|
|
spark.files.fetchTimeout |
- 60 | -- Communication timeout to use when fetching files added through SparkContext.addFile() from - the driver, in seconds. - | +spark.hadoop.cloneConf |
+ false | +If set to true, clones a new Hadoop Configuration object for each task. This
+ option should be enabled to work around Configuration thread-safety issues (see
+ SPARK-2546 for more details).
+ This is disabled by default in order to avoid unexpected performance regressions for jobs that
+ are not affected by these issues. |
+
spark.hadoop.validateOutputSpecs |
+ true | +If set to true, validates the output specification (e.g. checking if the output directory already exists) + used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing + output directories. We recommend that users do not disable this except if trying to achieve compatibility with + previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. + This setting is ignored for jobs generated through Spark Streaming's StreamingContext, since + data may need to be rewritten to pre-existing output directories during checkpoint recovery. | |||
spark.storage.memoryFraction |
@@ -689,6 +781,15 @@ Apart from these, the following properties are also available, and may be useful
increase it if you configure your own old generation size.
|||||
spark.storage.memoryMapThreshold |
+ 2097152 | ++ Size of a block, in bytes, above which Spark memory maps when reading a block from disk. + This prevents Spark from memory mapping very small blocks. In general, memory + mapping has high overhead for blocks close to or below the page size of the operating system. + | +|||
spark.storage.unrollFraction |
0.2 | @@ -707,15 +808,6 @@ Apart from these, the following properties are also available, and may be useful directories on Tachyon file system.||||
spark.storage.memoryMapThreshold |
- 2097152 | -- Size of a block, in bytes, above which Spark memory maps when reading a block from disk. - This prevents Spark from memory mapping very small blocks. In general, memory - mapping has high overhead for blocks close to or below the page size of the operating system. - | -|||
spark.tachyonStore.url |
tachyon://localhost:19998 | @@ -723,115 +815,53 @@ Apart from these, the following properties are also available, and may be useful The URL of the underlying Tachyon file system in the TachyonStore.||||
spark.cleaner.ttl |
- (infinite) | -- Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks - generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be - forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in - case of Spark Streaming applications). Note that any RDD that persists in memory for more than - this duration will be cleared as well. - | -|||
spark.hadoop.validateOutputSpecs |
- true | -If set to true, validates the output specification (e.g. checking if the output directory already exists) - used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing - output directories. We recommend that users do not disable this except if trying to achieve compatibility with - previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. - This setting is ignored for jobs generated through Spark Streaming's StreamingContext, since - data may need to be rewritten to pre-existing output directories during checkpoint recovery. | -|||
spark.hadoop.cloneConf |
- false | -If set to true, clones a new Hadoop Configuration object for each task. This
- option should be enabled to work around Configuration thread-safety issues (see
- SPARK-2546 for more details).
- This is disabled by default in order to avoid unexpected performance regressions for jobs that
- are not affected by these issues. |
-|||
spark.executor.heartbeatInterval |
- 10000 | -Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let - the driver know that the executor is still alive and update it with metrics for in-progress - tasks. | -
Property Name | Default | Meaning | ||
---|---|---|---|---|
spark.driver.host |
- (local hostname) | -- Hostname or IP address for the driver to listen on. - This is used for communicating with the executors and the standalone Master. - | -||
spark.driver.port |
- (random) | -- Port for the driver to listen on. - This is used for communicating with the executors and the standalone Master. - | -||
spark.fileserver.port |
- (random) | -- Port for the driver's HTTP file server to listen on. - | -||
spark.broadcast.port |
- (random) | -- Port for the driver's HTTP broadcast server to listen on. - This is not relevant for torrent broadcast. - | -||
spark.replClassServer.port |
- (random) | -- Port for the driver's HTTP class server to listen on. - This is only relevant for the Spark shell. - | -||
spark.blockManager.port |
- (random) | +spark.akka.failure-detector.threshold |
+ 300.0 | - Port for all block managers to listen on. These exist on both the driver and the executors. + This is set to a larger value to disable failure detector that comes inbuilt akka. It can be + enabled again, if you plan to use this feature (Not recommended). This maps to akka's + `akka.remote.transport-failure-detector.threshold`. Tune this in combination of + `spark.akka.heartbeat.pauses` and `spark.akka.heartbeat.interval` if you need to. |
spark.executor.port |
- (random) | +spark.akka.frameSize |
+ 10 |
- Port for the executor to listen on. This is used for communicating with the driver.
+ Maximum message size to allow in "control plane" communication (for serialized tasks and task
+ results), in MB. Increase this if your tasks need to send back large results to the driver
+ (e.g. using collect() on a large dataset).
|
spark.port.maxRetries |
- 16 | +spark.akka.heartbeat.interval |
+ 1000 | - Default maximum number of retries when binding to a port before giving up. + This is set to a larger value to disable the transport failure detector that comes built in to + Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger + interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more + informative for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` + if you need to. A likely positive use case for using failure detector would be: a sensistive + failure detector can help evict rogue executors quickly. However this is usually not the case + as GC pauses and network lags are expected in a real Spark cluster. Apart from that enabling + this leads to a lot of exchanges of heart beats between nodes leading to flooding the network + with those. |
spark.akka.frameSize |
- 10 | +spark.akka.heartbeat.pauses |
+ 6000 |
- Maximum message size to allow in "control plane" communication (for serialized tasks and task
- results), in MB. Increase this if your tasks need to send back large results to the driver
- (e.g. using collect() on a large dataset).
+ This is set to a larger value to disable the transport failure detector that comes built in to Akka.
+ It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart
+ beat pause in seconds for Akka. This can be used to control sensitivity to GC pauses. Tune
+ this along with `spark.akka.heartbeat.interval` if you need to.
|
spark.network.timeout |
- 120 | -
- Default timeout for all network interactions, in seconds. This config will be used in
- place of spark.core.connection.ack.wait.timeout , spark.akka.timeout ,
- spark.storage.blockManagerSlaveTimeoutMs or
- spark.shuffle.io.connectionTimeout , if they are not configured.
- |
-||
spark.akka.heartbeat.pauses |
- 6000 | -- This is set to a larger value to disable failure detector that comes inbuilt akka. It can be - enabled again, if you plan to use this feature (Not recommended). Acceptable heart beat pause - in seconds for akka. This can be used to control sensitivity to gc pauses. Tune this in - combination of `spark.akka.heartbeat.interval` and `spark.akka.failure-detector.threshold` - if you need to. - | -||
spark.akka.failure-detector.threshold |
- 300.0 | +spark.blockManager.port |
+ (random) | - This is set to a larger value to disable failure detector that comes inbuilt akka. It can be - enabled again, if you plan to use this feature (Not recommended). This maps to akka's - `akka.remote.transport-failure-detector.threshold`. Tune this in combination of - `spark.akka.heartbeat.pauses` and `spark.akka.heartbeat.interval` if you need to. + Port for all block managers to listen on. These exist on both the driver and the executors. |
spark.akka.heartbeat.interval |
- 1000 | +spark.broadcast.port |
+ (random) | - This is set to a larger value to disable failure detector that comes inbuilt akka. It can be - enabled again, if you plan to use this feature (Not recommended). A larger interval value in - seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for - akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and - `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using - failure detector can be, a sensistive failure detector can help evict rogue executors really - quick. However this is usually not the case as gc pauses and network lags are expected in a - real Spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats - between nodes leading to flooding the network with those. + Port for the driver's HTTP broadcast server to listen on. + This is not relevant for torrent broadcast. |
spark.shuffle.io.preferDirectBufs |
- true | +spark.driver.host |
+ (local hostname) | - (Netty only) Off-heap buffers are used to reduce garbage collection during shuffle and cache - block transfer. For environments where off-heap memory is tightly limited, users may wish to - turn this off to force all allocations from Netty to be on-heap. + Hostname or IP address for the driver to listen on. + This is used for communicating with the executors and the standalone Master. |
spark.shuffle.io.numConnectionsPerPeer |
- 1 | +spark.driver.port |
+ (random) | - (Netty only) Connections between hosts are reused in order to reduce connection buildup for - large clusters. For clusters with many hard disks and few hosts, this may result in insufficient - concurrency to saturate all disks, and so users may consider increasing this value. + Port for the driver to listen on. + This is used for communicating with the executors and the standalone Master. |
spark.shuffle.io.maxRetries |
- 3 | +spark.executor.port |
+ (random) | - (Netty only) Fetches that fail due to IO-related exceptions are automatically retried if this is - set to a non-zero value. This retry logic helps stabilize large shuffles in the face of long GC - pauses or transient network connectivity issues. + Port for the executor to listen on. This is used for communicating with the driver. |
spark.shuffle.io.retryWait |
- 5 | +spark.fileserver.port |
+ (random) |
- (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying
- is simply maxRetries * retryWait , by default 15 seconds.
+ Port for the driver's HTTP file server to listen on.
|
Property Name | Default | Meaning | ||
---|---|---|---|---|
spark.task.cpus |
- 1 | +spark.network.timeout |
+ 120 |
- Number of cores to allocate for each task.
+ Default timeout for all network interactions, in seconds. This config will be used in
+ place of spark.core.connection.ack.wait.timeout , spark.akka.timeout ,
+ spark.storage.blockManagerSlaveTimeoutMs or
+ spark.shuffle.io.connectionTimeout , if they are not configured.
|
spark.task.maxFailures |
- 4 | +spark.port.maxRetries |
+ 16 | - Number of individual task failures before giving up on the job. - Should be greater than or equal to 1. Number of allowed retries = this value - 1. + Default maximum number of retries when binding to a port before giving up. |
spark.scheduler.mode |
- FIFO | +spark.replClassServer.port |
+ (random) |
- The scheduling mode between
- jobs submitted to the same SparkContext. Can be set to FAIR
- to use fair sharing instead of queueing jobs one after another. Useful for
- multi-user services.
+ Port for the driver's HTTP class server to listen on.
+ This is only relevant for the Spark shell.
|
Property Name | Default | Meaning | ||
---|---|---|---|---|
spark.cores.max |
(not set) | @@ -973,43 +967,12 @@ Apart from these, the following properties are also available, and may be useful|||
spark.mesos.coarse |
- false | -- If set to "true", runs over Mesos clusters in - "coarse-grained" sharing mode, - where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per - Spark task. This gives lower-latency scheduling for short queries, but leaves resources in use - for the whole duration of the Spark job. - | -||
spark.speculation |
+ spark.localExecution.enabled |
false | - If set to "true", performs speculative execution of tasks. This means if one or more tasks are - running slowly in a stage, they will be re-launched. - | -|
spark.speculation.interval |
- 100 | -- How often Spark will check for tasks to speculate, in milliseconds. - | -||
spark.speculation.quantile |
- 0.75 | -- Percentage of tasks which must be complete before speculation is enabled for a particular stage. - | -||
spark.speculation.multiplier |
- 1.5 | -- How many times slower a task is than the median to be considered for speculation. + Enables Spark to run certain jobs, such as first() or take() on the driver, without sending + tasks to the cluster. This can make certain jobs execute very quickly, but may require + shipping a whole partition of data to the driver. | ||
spark.locality.wait.process |
+ spark.locality.wait.node |
spark.locality.wait | - Customize the locality wait for process locality. This affects tasks that attempt to access - cached data in a particular executor process. + Customize the locality wait for node locality. For example, you can set this to 0 to skip + node locality and search immediately for rack locality (if your cluster has rack information). | |
spark.locality.wait.node |
+ spark.locality.wait.process |
spark.locality.wait | - Customize the locality wait for node locality. For example, you can set this to 0 to skip - node locality and search immediately for rack locality (if your cluster has rack information). + Customize the locality wait for process locality. This affects tasks that attempt to access + cached data in a particular executor process. | |
spark.scheduler.revive.interval |
- 1000 | +spark.scheduler.maxRegisteredResourcesWaitingTime |
+ 30000 | - The interval length for the scheduler to revive the worker resource offers to run tasks + Maximum amount of time to wait for resources to register before scheduling begins (in milliseconds). |
spark.scheduler.minRegisteredResourcesRatio |
- 0.0 for Mesos and Standalone mode, 0.8 for YARN | +0.8 for YARN mode; 0.0 otherwise | The minimum ratio of registered resources (registered resources / total expected resources) (resources are executors in yarn mode, CPU cores in standalone mode) @@ -1068,25 +1031,70 @@ Apart from these, the following properties are also available, and may be useful | |
spark.scheduler.maxRegisteredResourcesWaitingTime |
- 30000 | +spark.scheduler.mode |
+ FIFO |
- Maximum amount of time to wait for resources to register before scheduling begins
+ The scheduling mode between
+ jobs submitted to the same SparkContext. Can be set to FAIR
+ to use fair sharing instead of queueing jobs one after another. Useful for
+ multi-user services.
+ |
+
spark.scheduler.revive.interval |
+ 1000 | ++ The interval length for the scheduler to revive the worker resource offers to run tasks (in milliseconds). | ||
spark.localExecution.enabled |
+ spark.speculation |
false | - Enables Spark to run certain jobs, such as first() or take() on the driver, without sending - tasks to the cluster. This can make certain jobs execute very quickly, but may require - shipping a whole partition of data to the driver. + If set to "true", performs speculative execution of tasks. This means if one or more tasks are + running slowly in a stage, they will be re-launched. + | +|
spark.speculation.interval |
+ 100 | ++ How often Spark will check for tasks to speculate, in milliseconds. + | +||
spark.speculation.multiplier |
+ 1.5 | ++ How many times slower a task is than the median to be considered for speculation. + | +||
spark.speculation.quantile |
+ 0.75 | ++ Percentage of tasks which must be complete before speculation is enabled for a particular stage. + | +||
spark.task.cpus |
+ 1 | ++ Number of cores to allocate for each task. + | +||
spark.task.maxFailures |
+ 4 | ++ Number of individual task failures before giving up on the job. + Should be greater than or equal to 1. Number of allowed retries = this value - 1. |
Property Name | Default | Meaning | |
---|---|---|---|
spark.dynamicAllocation.executorIdleTimeout |
+ 600 | ++ If dynamic allocation is enabled and an executor has been idle for more than this duration + (in seconds), the executor will be removed. For more detail, see this + description. | |
spark.dynamicAllocation.initialExecutors |
spark.dynamicAllocation.minExecutors |
- (none) | - Lower bound for the number of executors if dynamic allocation is enabled (required). + Initial number of executors to run if dynamic allocation is enabled. |
spark.dynamicAllocation.maxExecutors |
- (none) | +Integer.MAX_VALUE | ++ Upper bound for the number of executors if dynamic allocation is enabled. + | +
spark.dynamicAllocation.minExecutors |
+ 0 | - Upper bound for the number of executors if dynamic allocation is enabled (required). + Lower bound for the number of executors if dynamic allocation is enabled. | |
spark.dynamicAllocation.schedulerBacklogTimeout |
- 60 | +5 | If dynamic allocation is enabled and there have been pending tasks backlogged for more than this duration (in seconds), new executors will be requested. For more detail, see this @@ -1136,20 +1161,30 @@ Apart from these, the following properties are also available, and may be useful description. |
spark.dynamicAllocation.executorIdleTimeout |
- 600 | -- If dynamic allocation is enabled and an executor has been idle for more than this duration - (in seconds), the executor will be removed. For more detail, see this - description. - | -
Property Name | Default | Meaning | ||
---|---|---|---|---|
spark.acls.enable |
+ false | ++ Whether Spark acls should are enabled. If enabled, this checks to see if the user has + access permissions to view or modify the job. Note this requires the user to be known, + so if the user comes across as null no checks are done. Filters can be used with the UI + to authenticate and set the user. + | +||
spark.admin.acls |
+ Empty | ++ Comma separated list of users/administrators that have view and modify access to all Spark jobs. + This can be used if you run on a shared cluster and have a set of administrators or devs who + help debug when things work. + | +||
spark.authenticate |
false | @@ -1166,6 +1201,15 @@ Apart from these, the following properties are also available, and may be useful not running on YARN and authentication is enabled.|||
spark.core.connection.ack.wait.timeout |
+ 60 | ++ Number of seconds for the connection to wait for ack to occur before timing + out and giving up. To avoid unwilling timeout caused by long pause like GC, + you can set larger value. + | +||
spark.core.connection.auth.wait.timeout |
30 | @@ -1175,12 +1219,11 @@ Apart from these, the following properties are also available, and may be useful|||
spark.core.connection.ack.wait.timeout |
- 60 | +spark.modify.acls |
+ Empty | - Number of seconds for the connection to wait for ack to occur before timing - out and giving up. To avoid unwilling timeout caused by long pause like GC, - you can set larger value. + Comma separated list of users that have modify access to the Spark job. By default only the + user that started the Spark job has access to modify it (kill it for example). |
spark.acls.enable |
- false | -- Whether Spark acls should are enabled. If enabled, this checks to see if the user has - access permissions to view or modify the job. Note this requires the user to be known, - so if the user comes across as null no checks are done. Filters can be used with the UI - to authenticate and set the user. - | -||
spark.ui.view.acls |
Empty | @@ -1215,25 +1248,88 @@ Apart from these, the following properties are also available, and may be useful user that started the Spark job has view access.|||
spark.modify.acls |
- Empty | -- Comma separated list of users that have modify access to the Spark job. By default only the - user that started the Spark job has access to modify it (kill it for example). - | -||
spark.admin.acls |
- Empty | -- Comma separated list of users/administrators that have view and modify access to all Spark jobs. - This can be used if you run on a shared cluster and have a set of administrators or devs who - help debug when things work. - | -
Property Name | Default | Meaning |
---|---|---|
spark.ssl.enabled |
+ false | +
+ Whether to enable SSL connections on all supported protocols. + +All the SSL settings like Use |
+
spark.ssl.enabledAlgorithms |
+ Empty | ++ A comma separated list of ciphers. The specified ciphers must be supported by JVM. + The reference list of protocols one can find on + this + page. + | +
spark.ssl.keyPassword |
+ None | ++ A password to the private key in key-store. + | +
spark.ssl.keyStore |
+ None | ++ A path to a key-store file. The path can be absolute or relative to the directory where + the component is started in. + | +
spark.ssl.keyStorePassword |
+ None | ++ A password to the key-store. + | +
spark.ssl.protocol |
+ None | ++ A protocol name. The protocol must be supported by JVM. The reference list of protocols + one can find on this + page. + | +
spark.ssl.trustStore |
+ None | ++ A path to a trust-store file. The path can be absolute or relative to the directory + where the component is started in. + | +
spark.ssl.trustStorePassword |
+ None | ++ A password to the trust-store. + | +
Property Name | Default | Meaning | |
---|---|---|---|
spark.streaming.receiver.maxRate |
- infinite | +not set | - Maximum number records per second at which each receiver will receive data. + Maximum rate (number of records per second) at which each receiver will receive data. Effectively, each stream will consume at most this number of records per second. Setting this configuration to 0 or a negative number will put no limit on the rate. See the deployment guide @@ -1279,15 +1375,27 @@ Apart from these, the following properties are also available, and may be useful higher memory usage in Spark. |
spark.streaming.kafka.maxRatePerPartition |
+ not set | ++ Maximum rate (number of records per second) at which data will be read from each Kafka + partition when using the new Kafka direct stream API. See the + Kafka Integration guide + for more details. + | +
-
-
-
- > transactions = ...
+
+FPGrowth fpg = new FPGrowth()
+ .setMinSupport(0.2)
+ .setNumPartitions(10);
+FPGrowthModel
Note: modifying variables other than Accumulators outside of the
foreach()
may result in undefined behavior. See Understanding closures for more details.spark.mesos.coarse
spark.mesos.executor.home
SPARK_HOME
spark.executor.uri
.
spark.mesos.executor.memoryOverhead
spark.executor.memory
, specified in MiB,
+ which is used to calculate the total Mesos task memory. A value of 384
+ implies a 384MiB overhead. Additionally, there is a hard-coded 7% minimum
+ overhead. The final overhead will be the larger of either
+ `spark.mesos.executor.memoryOverhead` or 7% of `spark.executor.memory`.
spark.executor.instances
spark.dynamicAllocation.enabled
.
+ spark.yarn.executor.memoryOverhead
Scala/Java | Python | Meaning |
---|---|---|
SaveMode.ErrorIfExists (default) |
+ "error" (default) |
+ + When saving a DataFrame to a data source, if data already exists, + an exception is expected to be thrown. + | +
SaveMode.Append |
+ "append" |
+ + When saving a DataFrame to a data source, if data/table already exists, + contents of the DataFrame are expected to be appended to existing data. + | +
SaveMode.Overwrite |
+ "overwrite" |
+ + Overwrite mode means that when saving a DataFrame to a data source, + if data/table already exists, existing data is expected to be overwritten by the contents of + the DataFrame. + | +
SaveMode.Ignore |
+ "ignore" |
+ + Ignore mode means that when saving a DataFrame to a data source, if data already exists, + the save operation is expected to not save the contents of the DataFrame and to not + change the existing data. This is similar to a `CREATE TABLE IF NOT EXISTS` in SQL. + | +
spark.sql.parquet.int96AsTimestamp |
+ true | ++ Some Parquet-producing systems, in particular Impala, store Timestamp into INT96. Spark would also + store Timestamp as INT96 because we need to avoid precision lost of the nanoseconds field. This + flag tells Spark SQL to interpret INT96 data as a timestamp to provide compatibility with these systems. + | +|||||||||||||||||||||
spark.sql.parquet.cacheMetadata |
true | @@ -619,8 +1094,8 @@ Configuration of Parquet can be done using the `setConf` method on SQLContext or
Property Name | Meaning |
---|---|
url |
+ + The JDBC URL to connect to. + | +
dbtable |
+ + The JDBC table that should be read. Note that anything that is valid in a `FROM` clause of + a SQL query can be used. For example, instead of a full table you could also use a + subquery in parentheses. + | +
driver |
+ + The class name of the JDBC driver needed to connect to this URL. This class with be loaded + on the master and workers before running an JDBC commands to allow the driver to + register itself with the JDBC subsystem. + | +
partitionColumn, lowerBound, upperBound, numPartitions |
+
+ These options must all be specified if any of them is specified. They describe how to
+ partition the table when reading in parallel from multiple workers.
+ partitionColumn must be a numeric column from the table in question.
+ |
+
DecimalType | -scala.math.BigDecimal | +java.math.BigDecimal | DecimalType | @@ -1447,7 +2102,7 @@ please use factory methods provided in||
StructType | -org.apache.spark.sql.api.java.Row | +org.apache.spark.sql.Row |
DataTypes.createStructType(fields) Note: fields is a List or an array of StructFields. @@ -1468,10 +2123,10 @@ please use factory methods provided in
-All data types of Spark SQL are located in the package of `pyspark.sql`.
+All data types of Spark SQL are located in the package of `pyspark.sql.types`.
You can access them by doing
{% highlight python %}
-from pyspark.sql import *
+from pyspark.sql.types import *
{% endhighlight %}
|
- Spark 1.2 without write ahead log + Spark 1.1 or earlier, OR
+ Spark 1.2 or later without write ahead logs
- Zero data loss with reliable receivers and files
+ Zero data loss with reliable receivers
+ At-least once semantics
Past data lost with all receivers
- Zero data loss with files -
+ At-least once semantics +
+ At-least once semantics +
+ * bin/run-example ml.JavaDeveloperApiExample + *+ */ +public class JavaDeveloperApiExample { + + public static void main(String[] args) throws Exception { + SparkConf conf = new SparkConf().setAppName("JavaDeveloperApiExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + SQLContext jsql = new SQLContext(jsc); + + // Prepare training data. + List
- , Optional
- "spark-submit": if class is "org.apache.spark.deploy.SparkSubmit", the + * {@link SparkLauncher} class is used to launch a Spark application. + *
- "spark-class": if another class is provided, an internal Spark class is run. + *
- , Optional
-
+ *