diff --git a/bin/spark-class b/bin/spark-class index 76fde3e448891..1b0d309cc5b1c 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -47,9 +47,9 @@ DEFAULT_MEM=${SPARK_MEM:-512m} SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" -# Add java opts and memory settings for master, worker, executors, and repl. +# Add java opts and memory settings for master, worker, history server, executors, and repl. case "$1" in - # Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. + # Master, Worker, and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. 'org.apache.spark.deploy.master.Master') OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_MASTER_OPTS" OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} @@ -58,6 +58,10 @@ case "$1" in OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_WORKER_OPTS" OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} ;; + 'org.apache.spark.deploy.history.HistoryServer') + OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_HISTORY_OPTS" + OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} + ;; # Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. 'org.apache.spark.executor.CoarseGrainedExecutorBackend') diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index f488cfdbeceb6..4302c1b6b7ff4 100755 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -45,14 +45,17 @@ if "x%OUR_JAVA_MEM%"=="x" set OUR_JAVA_MEM=512m set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true -rem Add java opts and memory settings for master, worker, executors, and repl. -rem Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. +rem Add java opts and memory settings for master, worker, history server, executors, and repl. +rem Master, Worker and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. if "%1"=="org.apache.spark.deploy.master.Master" ( set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_MASTER_OPTS% if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% ) else if "%1"=="org.apache.spark.deploy.worker.Worker" ( set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_WORKER_OPTS% if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% +) else if "%1"=="org.apache.spark.deploy.history.HistoryServer" ( + set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_HISTORY_OPTS% + if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% rem Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. ) else if "%1"=="org.apache.spark.executor.CoarseGrainedExecutorBackend" ( diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index e5e15617acb10..9d429dceeb858 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows * logging messages at different levels using methods that only evaluate parameters lazily if the * log level is enabled. - * + * * NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility. * This will likely be changed or removed in future releases. */ @@ -60,7 +60,7 @@ trait Logging { protected def logDebug(msg: => String) { if (log.isDebugEnabled) log.debug(msg) } - + protected def logTrace(msg: => String) { if (log.isTraceEnabled) log.trace(msg) } @@ -117,10 +117,10 @@ trait Logging { val defaultLogProps = "org/apache/spark/log4j-defaults.properties" val classLoader = this.getClass.getClassLoader Option(classLoader.getResource(defaultLogProps)) match { - case Some(url) => + case Some(url) => PropertyConfigurator.configure(url) log.info(s"Using Spark's default log4j profile: $defaultLogProps") - case None => + case None => System.err.println(s"Spark was unable to load $defaultLogProps") } } @@ -135,4 +135,16 @@ trait Logging { private object Logging { @volatile private var initialized = false val initLock = new Object() + try { + // We use reflection here to handle the case where users remove the + // slf4j-to-jul bridge order to route their logs to JUL. + val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler") + bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) + val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] + if (!installed) { + bridgeClass.getMethod("install").invoke(null) + } + } catch { + case e: ClassNotFoundException => // can't log anything yet so just fail silently + } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index be56a0fd77f6a..c9af0778bdb29 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -219,15 +219,12 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val eventLogger: Option[EventLoggingListener] = { if (conf.getBoolean("spark.eventLog.enabled", false)) { val logger = new EventLoggingListener(appName, conf) + logger.start() listenerBus.addListener(logger) Some(logger) } else None } - // Information needed to replay logged events, if any - private[spark] val eventLoggingInfo: Option[EventLoggingInfo] = - eventLogger.map { logger => Some(logger.info) }.getOrElse(None) - // At this point, all relevant SparkListeners have been registered, so begin releasing events listenerBus.start() @@ -292,6 +289,7 @@ class SparkContext(config: SparkConf) extends Logging { cleaner.foreach(_.start()) postEnvironmentUpdate() + postApplicationStart() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration: Configuration = { @@ -777,6 +775,9 @@ class SparkContext(config: SparkConf) extends Logging { listenerBus.addListener(listener) } + /** The version of Spark on which this application is running. */ + def version = SparkContext.SPARK_VERSION + /** * Return a map from the slave to the max memory available for caching and the remaining * memory available for caching. @@ -930,8 +931,8 @@ class SparkContext(config: SparkConf) extends Logging { /** Shut down the SparkContext. */ def stop() { + postApplicationEnd() ui.stop() - eventLogger.foreach(_.stop()) // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. val dagSchedulerCopy = dagScheduler @@ -940,13 +941,14 @@ class SparkContext(config: SparkConf) extends Logging { metadataCleaner.cancel() cleaner.foreach(_.stop()) dagSchedulerCopy.stop() - listenerBus.stop() taskScheduler = null // TODO: Cache.stop()? env.stop() SparkEnv.set(null) ShuffleMapTask.clearCache() ResultTask.clearCache() + listenerBus.stop() + eventLogger.foreach(_.stop()) logInfo("Successfully stopped SparkContext") } else { logInfo("SparkContext already stopped") @@ -1175,6 +1177,20 @@ class SparkContext(config: SparkConf) extends Logging { /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() + /** Post the application start event */ + private def postApplicationStart() { + listenerBus.post(SparkListenerApplicationStart(appName, startTime, sparkUser)) + } + + /** + * Post the application end event to all listeners immediately, rather than adding it + * to the event queue for it to be asynchronously processed eventually. Otherwise, a race + * condition exists in which the listeners may stop before this event has been propagated. + */ + private def postApplicationEnd() { + listenerBus.post(SparkListenerApplicationEnd(System.currentTimeMillis)) + } + /** Post the environment update event once the task scheduler is ready */ private def postEnvironmentUpdate() { if (taskScheduler != null) { @@ -1200,6 +1216,8 @@ class SparkContext(config: SparkConf) extends Logging { */ object SparkContext extends Logging { + private[spark] val SPARK_VERSION = "1.0.0" + private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description" private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id" diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 15fa8a7679874..86305d2ea8a09 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy -import org.apache.spark.scheduler.EventLoggingInfo - private[spark] class ApplicationDescription( val name: String, val maxCores: Option[Int], @@ -26,7 +24,7 @@ private[spark] class ApplicationDescription( val command: Command, val sparkHome: Option[String], var appUiUrl: String, - val eventLogInfo: Option[EventLoggingInfo] = None) + val eventLogDir: Option[String] = None) extends Serializable { val user = System.getProperty("user.name", "") 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 new file mode 100644 index 0000000000000..16abfe920da72 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -0,0 +1,292 @@ +/* + * 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.{FileStatus, Path} +import org.eclipse.jetty.servlet.ServletContextHandler + +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.scheduler._ +import org.apache.spark.ui.{WebUI, SparkUI} +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.Utils + +/** + * A web server that renders SparkUIs of completed applications. + * + * For the standalone mode, MasterWebUI already achieves this functionality. Thus, the + * main use case of the HistoryServer is in other deploy modes (e.g. Yarn or Mesos). + * + * The logging directory structure is as follows: Within the given base directory, each + * application's event logs are maintained in the application's own sub-directory. This + * is the same structure as maintained in the event log write code path in + * EventLoggingListener. + * + * @param baseLogDir The base directory in which event logs are found + */ +class HistoryServer( + val baseLogDir: String, + conf: SparkConf) + extends WebUI(new SecurityManager(conf)) with Logging { + + import HistoryServer._ + + private val fileSystem = Utils.getHadoopFileSystem(baseLogDir) + private val localHost = Utils.localHostName() + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) + private val port = WEB_UI_PORT + + // A timestamp of when the disk was last accessed to check for log updates + private var lastLogCheckTime = -1L + + // Number of completed applications found in this directory + private var numCompletedApplications = 0 + + @volatile private var stopped = false + + /** + * A background thread that periodically checks for event log updates on disk. + * + * If a log check is invoked manually in the middle of a period, this thread re-adjusts the + * time at which it performs the next log check to maintain the same period as before. + * + * TODO: Add a mechanism to update manually. + */ + private val logCheckingThread = new Thread { + override def run() { + while (!stopped) { + val now = System.currentTimeMillis + if (now - lastLogCheckTime > UPDATE_INTERVAL_MS) { + checkForLogs() + Thread.sleep(UPDATE_INTERVAL_MS) + } else { + // If the user has manually checked for logs recently, wait until + // UPDATE_INTERVAL_MS after the last check time + Thread.sleep(lastLogCheckTime + UPDATE_INTERVAL_MS - now) + } + } + } + } + + // A mapping of application ID to its history information, which includes the rendered UI + val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]() + + /** + * Start the history server. + * + * This starts a background thread that periodically synchronizes information displayed on + * this UI with the event logs in the provided base directory. + */ + def start() { + attachPage(new IndexPage(this)) + attachHandler(createStaticHandler(STATIC_RESOURCE_DIR, "/static")) + logCheckingThread.start() + } + + /** Bind to the HTTP server behind this web interface. */ + def bind() { + try { + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) + logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort)) + } catch { + case e: Exception => + logError("Failed to bind HistoryServer", e) + System.exit(1) + } + } + + /** + * Check for any updates to event logs in the base directory. This is only effective once + * the server has been bound. + * + * If a new completed application is found, the server renders the associated SparkUI + * from the application's event logs, attaches this UI to itself, and stores metadata + * information for this application. + * + * If the logs for an existing completed application are no longer found, the server + * removes all associated information and detaches the SparkUI. + */ + def checkForLogs() = synchronized { + if (serverInfo.isDefined) { + lastLogCheckTime = System.currentTimeMillis + logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTime)) + try { + val logStatus = fileSystem.listStatus(new Path(baseLogDir)) + val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() + val logInfos = logDirs + .sortBy { dir => getModificationTime(dir) } + .map { dir => (dir, EventLoggingListener.parseLoggingInfo(dir.getPath, fileSystem)) } + .filter { case (dir, info) => info.applicationComplete } + + // Logging information for applications that should be retained + val retainedLogInfos = logInfos.takeRight(RETAINED_APPLICATIONS) + val retainedAppIds = retainedLogInfos.map { case (dir, _) => dir.getPath.getName } + + // Remove any applications that should no longer be retained + appIdToInfo.foreach { case (appId, info) => + if (!retainedAppIds.contains(appId)) { + detachUI(info.ui) + appIdToInfo.remove(appId) + } + } + + // Render the application's UI if it is not already there + retainedLogInfos.foreach { case (dir, info) => + val appId = dir.getPath.getName + if (!appIdToInfo.contains(appId)) { + renderSparkUI(dir, info) + } + } + + // Track the total number of completed applications observed this round + numCompletedApplications = logInfos.size + + } catch { + case t: Throwable => logError("Exception in checking for event log updates", t) + } + } else { + logWarning("Attempted to check for event log updates before binding the server.") + } + } + + /** + * Render a new SparkUI from the event logs if the associated application is completed. + * + * HistoryServer looks for a special file that indicates application completion in the given + * directory. If this file exists, the associated application is regarded to be completed, in + * which case the server proceeds to render the SparkUI. Otherwise, the server does nothing. + */ + private def renderSparkUI(logDir: FileStatus, logInfo: EventLoggingInfo) { + val path = logDir.getPath + val appId = path.getName + val replayBus = new ReplayListenerBus(logInfo.logPaths, fileSystem, logInfo.compressionCodec) + val ui = new SparkUI(conf, replayBus, appId, "/history/" + appId) + val appListener = new ApplicationEventListener + replayBus.addListener(appListener) + + // Do not call ui.bind() to avoid creating a new server for each application + ui.start() + replayBus.replay() + if (appListener.applicationStarted) { + attachUI(ui) + val appName = appListener.appName + val sparkUser = appListener.sparkUser + val startTime = appListener.startTime + val endTime = appListener.endTime + val lastUpdated = getModificationTime(logDir) + ui.setAppName(appName + " (completed)") + appIdToInfo(appId) = ApplicationHistoryInfo(appId, appName, startTime, endTime, + lastUpdated, sparkUser, path, ui) + } + } + + /** Stop the server and close the file system. */ + override def stop() { + super.stop() + stopped = true + fileSystem.close() + } + + /** Attach a reconstructed UI to this server. Only valid after bind(). */ + private def attachUI(ui: SparkUI) { + assert(serverInfo.isDefined, "HistoryServer must be bound before attaching SparkUIs") + ui.getHandlers.foreach(attachHandler) + } + + /** Detach a reconstructed UI from this server. Only valid after bind(). */ + private def detachUI(ui: SparkUI) { + assert(serverInfo.isDefined, "HistoryServer must be bound before detaching SparkUIs") + ui.getHandlers.foreach(detachHandler) + } + + /** Return the address of this server. */ + def getAddress: String = "http://" + publicHost + ":" + boundPort + + /** Return the number of completed applications found, whether or not the UI is rendered. */ + def getNumApplications: Int = numCompletedApplications + + /** Return when this directory was last modified. */ + private def getModificationTime(dir: FileStatus): Long = { + try { + val logFiles = fileSystem.listStatus(dir.getPath) + if (logFiles != null && !logFiles.isEmpty) { + logFiles.map(_.getModificationTime).max + } else { + dir.getModificationTime + } + } catch { + case t: Throwable => + logError("Exception in accessing modification time of %s".format(dir.getPath), t) + -1L + } + } +} + +/** + * The recommended way of starting and stopping a HistoryServer is through the scripts + * start-history-server.sh and stop-history-server.sh. The path to a base log directory + * is must be specified, while the requested UI port is optional. For example: + * + * ./sbin/spark-history-server.sh /tmp/spark-events + * ./sbin/spark-history-server.sh hdfs://1.2.3.4:9000/spark-events + * + * This launches the HistoryServer as a Spark daemon. + */ +object HistoryServer { + private val conf = new SparkConf + + // Interval between each check for event log updates + val UPDATE_INTERVAL_MS = conf.getInt("spark.history.updateInterval", 10) * 1000 + + // How many applications to retain + val RETAINED_APPLICATIONS = conf.getInt("spark.history.retainedApplications", 250) + + // The port to which the web UI is bound + val WEB_UI_PORT = conf.getInt("spark.history.ui.port", 18080) + + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR + + def main(argStrings: Array[String]) { + val args = new HistoryServerArguments(argStrings) + val server = new HistoryServer(args.logDir, conf) + server.bind() + server.start() + + // Wait until the end of the world... or if the HistoryServer process is manually stopped + while(true) { Thread.sleep(Int.MaxValue) } + server.stop() + } +} + + +private[spark] case class ApplicationHistoryInfo( + id: String, + name: String, + startTime: Long, + endTime: Long, + lastUpdated: Long, + sparkUser: String, + logDirPath: Path, + ui: SparkUI) { + def started = startTime != -1 + def completed = endTime != -1 +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala new file mode 100644 index 0000000000000..943c061743dbd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -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.deploy.history + +import java.net.URI + +import org.apache.hadoop.fs.Path + +import org.apache.spark.util.Utils + +/** + * Command-line parser for the master. + */ +private[spark] class HistoryServerArguments(args: Array[String]) { + var logDir = "" + + parse(args.toList) + + private def parse(args: List[String]): Unit = { + args match { + case ("--dir" | "-d") :: value :: tail => + logDir = value + parse(tail) + + case ("--help" | "-h") :: tail => + printUsageAndExit(0) + + case Nil => + + case _ => + printUsageAndExit(1) + } + validateLogDir() + } + + private def validateLogDir() { + if (logDir == "") { + System.err.println("Logging directory must be specified.") + printUsageAndExit(1) + } + val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) + val path = new Path(logDir) + if (!fileSystem.exists(path)) { + System.err.println("Logging directory specified does not exist: %s".format(logDir)) + printUsageAndExit(1) + } + if (!fileSystem.getFileStatus(path).isDir) { + System.err.println("Logging directory specified is not a directory: %s".format(logDir)) + printUsageAndExit(1) + } + } + + private def printUsageAndExit(exitCode: Int) { + System.err.println( + "Usage: HistoryServer [options]\n" + + "\n" + + "Options:\n" + + " -d DIR, --dir DIR Location of event log files") + System.exit(exitCode) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala new file mode 100644 index 0000000000000..eeb22ab000558 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.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.deploy.history + +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.ui.{UIPage, UIUtils} + +private[spark] class IndexPage(parent: HistoryServer) extends UIPage("") { + + override def render(request: HttpServletRequest): Seq[Node] = { + val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated } + val appTable = UIUtils.listingTable(appHeader, appRow, appRows) + val content = +
+
+
    +
  • Event Log Location: {parent.baseLogDir}
  • +
+ { + if (parent.appIdToInfo.size > 0) { +

+ Showing {parent.appIdToInfo.size}/{parent.getNumApplications} + Completed Application{if (parent.getNumApplications > 1) "s" else ""} +

++ + appTable + } else { +

No Completed Applications Found

+ } + } +
+
+ UIUtils.basicSparkPage(content, "History Server") + } + + private val appHeader = Seq( + "App Name", + "Started", + "Completed", + "Duration", + "Spark User", + "Log Directory", + "Last Updated") + + private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { + val appName = if (info.started) info.name else info.logDirPath.getName + val uiAddress = parent.getAddress + info.ui.basePath + val startTime = if (info.started) UIUtils.formatDate(info.startTime) else "Not started" + val endTime = if (info.completed) UIUtils.formatDate(info.endTime) else "Not completed" + val difference = if (info.started && info.completed) info.endTime - info.startTime else -1L + val duration = if (difference > 0) UIUtils.formatDuration(difference) else "---" + val sparkUser = if (info.started) info.sparkUser else "Unknown user" + val logDirectory = info.logDirPath.getName + val lastUpdated = UIUtils.formatDate(info.lastUpdated) + + {appName} + {startTime} + {endTime} + {duration} + {sparkUser} + {logDirectory} + {lastUpdated} + + } +} 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 a58e14bdcdd34..9c90c4b4d11ef 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 @@ -29,6 +29,7 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension +import org.apache.hadoop.fs.FileSystem import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState} @@ -37,7 +38,7 @@ 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.metrics.MetricsSystem -import org.apache.spark.scheduler.ReplayListenerBus +import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{AkkaUtils, Utils} @@ -45,7 +46,8 @@ private[spark] class Master( host: String, port: Int, webUiPort: Int, - val securityMgr: SecurityManager) extends Actor with Logging { + val securityMgr: SecurityManager) + extends Actor with Logging { import context.dispatcher // to use Akka's scheduler.schedule() @@ -71,6 +73,7 @@ private[spark] class Master( var nextAppNumber = 0 val appIdToUI = new HashMap[String, SparkUI] + val fileSystemsUsed = new HashSet[FileSystem] val drivers = new HashSet[DriverInfo] val completedDrivers = new ArrayBuffer[DriverInfo] @@ -150,6 +153,7 @@ private[spark] class Master( override def postStop() { webUi.stop() + fileSystemsUsed.foreach(_.close()) masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() @@ -631,11 +635,7 @@ private[spark] class Master( waitingApps -= app // If application events are logged, use them to rebuild the UI - startPersistedSparkUI(app).map { ui => - app.desc.appUiUrl = ui.basePath - appIdToUI(app.id) = ui - webUi.attachUI(ui) - }.getOrElse { + if (!rebuildSparkUI(app)) { // Avoid broken links if the UI is not reconstructed app.desc.appUiUrl = "" } @@ -655,30 +655,35 @@ private[spark] class Master( } /** - * Start a new SparkUI rendered from persisted storage. If this is unsuccessful for any reason, - * return None. Otherwise return the reconstructed UI. + * Rebuild a new SparkUI from the given application's event logs. + * Return whether this is successful. */ - def startPersistedSparkUI(app: ApplicationInfo): Option[SparkUI] = { + def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name - val eventLogInfo = app.desc.eventLogInfo.getOrElse { return None } - val eventLogDir = eventLogInfo.logDir - val eventCompressionCodec = eventLogInfo.compressionCodec - val appConf = new SparkConf - eventCompressionCodec.foreach { codec => - appConf.set("spark.eventLog.compress", "true") - appConf.set("spark.io.compression.codec", codec) - } - val replayerBus = new ReplayListenerBus(appConf) - val ui = new SparkUI( - appConf, - replayerBus, - "%s (finished)".format(appName), - "/history/%s".format(app.id)) - - // Do not call ui.bind() to avoid creating a new server for each application - ui.start() - val success = replayerBus.replay(eventLogDir) - if (success) Some(ui) else None + val eventLogDir = app.desc.eventLogDir.getOrElse { return false } + val fileSystem = Utils.getHadoopFileSystem(eventLogDir) + val eventLogInfo = EventLoggingListener.parseLoggingInfo(eventLogDir, fileSystem) + val eventLogPaths = eventLogInfo.logPaths + val compressionCodec = eventLogInfo.compressionCodec + if (!eventLogPaths.isEmpty) { + try { + val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) + val ui = new SparkUI( + new SparkConf, replayBus, appName + " (completed)", "/history/" + app.id) + ui.start() + replayBus.replay() + app.desc.appUiUrl = ui.basePath + appIdToUI(app.id) = ui + webUi.attachUI(ui) + return true + } catch { + case t: Throwable => + logError("Exception in replaying log for application %s (%s)".format(appName, app.id), t) + } + } else { + logWarning("Application %s (%s) has no valid logs: %s".format(appName, app.id, eventLogDir)) + } + false } /** Generate a new app ID given a app's submission date */ 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 95fbb5be14cba..d0f1a9bc9ffd1 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 @@ -17,6 +17,8 @@ package org.apache.spark.deploy.master.ui +import javax.servlet.http.HttpServletRequest + import org.apache.spark.Logging import org.apache.spark.deploy.master.Master import org.apache.spark.ui.{SparkUI, WebUI} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 39ca920b7d935..c1fdc5cea173c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -35,7 +35,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I private val host = Utils.localHostName() private val port = requestedPort.getOrElse( - worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) + worker.conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) val timeout = AkkaUtils.askTimeout(worker.conf) /** Initialize all components of the server. */ @@ -63,6 +63,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } private[spark] object WorkerWebUI { + val DEFAULT_PORT = 8081 val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR - val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala new file mode 100644 index 0000000000000..affda13df6531 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.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.scheduler + +/** + * A simple listener for application events. + * + * This listener expects to hear events from a single application only. If events + * from multiple applications are seen, the behavior is unspecified. + */ +private[spark] class ApplicationEventListener extends SparkListener { + var appName = "" + var sparkUser = "" + var startTime = -1L + var endTime = -1L + + def applicationStarted = startTime != -1 + + def applicationFinished = endTime != -1 + + def applicationDuration: Long = { + val difference = endTime - startTime + if (applicationStarted && applicationFinished && difference > 0) difference else -1L + } + + override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { + appName = applicationStart.appName + startTime = applicationStart.time + sparkUser = applicationStart.sparkUser + } + + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { + endTime = applicationEnd.time + } +} 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 217f8825c2ae9..b983c16af14f4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -17,11 +17,14 @@ package org.apache.spark.scheduler +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileSystem, Path} import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{JsonProtocol, FileLogger} +import org.apache.spark.util.{FileLogger, JsonProtocol} /** * A SparkListener that logs events to persistent storage. @@ -36,6 +39,8 @@ import org.apache.spark.util.{JsonProtocol, FileLogger} private[spark] class EventLoggingListener(appName: String, conf: SparkConf) extends SparkListener with Logging { + import EventLoggingListener._ + private val shouldCompress = conf.getBoolean("spark.eventLog.compress", false) private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", false) private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024 @@ -46,17 +51,21 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) private val logger = new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite) - // Information needed to replay the events logged by this listener later - val info = { - val compressionCodec = if (shouldCompress) { - Some(conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC)) - } else None - EventLoggingInfo(logDir, compressionCodec) + /** + * Begin logging events. + * If compression is used, log a file that indicates which compression library is used. + */ + def start() { + logInfo("Logging events to %s".format(logDir)) + if (shouldCompress) { + val codec = conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) + logger.newFile(COMPRESSION_CODEC_PREFIX + codec) + } + logger.newFile(SPARK_VERSION_PREFIX + SparkContext.SPARK_VERSION) + logger.newFile(LOG_PREFIX + logger.fileIndex) } - logInfo("Logging events to %s".format(logDir)) - - /** Log the event as JSON */ + /** Log the event as JSON. */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { val eventJson = compact(render(JsonProtocol.sparkEventToJson(event))) logger.logLine(eventJson) @@ -90,9 +99,118 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) logEvent(event, flushLogger = true) override def onUnpersistRDD(event: SparkListenerUnpersistRDD) = logEvent(event, flushLogger = true) + override def onApplicationStart(event: SparkListenerApplicationStart) = + logEvent(event, flushLogger = true) + override def onApplicationEnd(event: SparkListenerApplicationEnd) = + logEvent(event, flushLogger = true) + + /** + * Stop logging events. + * In addition, create an empty special file to indicate application completion. + */ + def stop() = { + logger.newFile(APPLICATION_COMPLETE) + logger.stop() + } +} + +private[spark] object EventLoggingListener extends Logging { + val LOG_PREFIX = "EVENT_LOG_" + val SPARK_VERSION_PREFIX = "SPARK_VERSION_" + val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" + val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" + + // A cache for compression codecs to avoid creating the same codec many times + private val codecMap = new mutable.HashMap[String, CompressionCodec] + + def isEventLogFile(fileName: String): Boolean = { + fileName.startsWith(LOG_PREFIX) + } + + def isSparkVersionFile(fileName: String): Boolean = { + fileName.startsWith(SPARK_VERSION_PREFIX) + } + + def isCompressionCodecFile(fileName: String): Boolean = { + fileName.startsWith(COMPRESSION_CODEC_PREFIX) + } + + def isApplicationCompleteFile(fileName: String): Boolean = { + fileName == APPLICATION_COMPLETE + } + + def parseSparkVersion(fileName: String): String = { + if (isSparkVersionFile(fileName)) { + fileName.replaceAll(SPARK_VERSION_PREFIX, "") + } else "" + } + + def parseCompressionCodec(fileName: String): String = { + if (isCompressionCodecFile(fileName)) { + fileName.replaceAll(COMPRESSION_CODEC_PREFIX, "") + } else "" + } + + /** + * Parse the event logging information associated with the logs in the given directory. + * + * Specifically, this looks for event log files, the Spark version file, the compression + * codec file (if event logs are compressed), and the application completion file (if the + * application has run to completion). + */ + def parseLoggingInfo(logDir: Path, fileSystem: FileSystem): EventLoggingInfo = { + try { + val fileStatuses = fileSystem.listStatus(logDir) + val filePaths = + if (fileStatuses != null) { + fileStatuses.filter(!_.isDir).map(_.getPath).toSeq + } else { + Seq[Path]() + } + if (filePaths.isEmpty) { + logWarning("No files found in logging directory %s".format(logDir)) + } + EventLoggingInfo( + logPaths = filePaths.filter { path => isEventLogFile(path.getName) }, + sparkVersion = filePaths + .find { path => isSparkVersionFile(path.getName) } + .map { path => parseSparkVersion(path.getName) } + .getOrElse(""), + compressionCodec = filePaths + .find { path => isCompressionCodecFile(path.getName) } + .map { path => + val codec = EventLoggingListener.parseCompressionCodec(path.getName) + val conf = new SparkConf + conf.set("spark.io.compression.codec", codec) + codecMap.getOrElseUpdate(codec, CompressionCodec.createCodec(conf)) + }, + applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) } + ) + } catch { + case t: Throwable => + logError("Exception in parsing logging info from directory %s".format(logDir), t) + EventLoggingInfo.empty + } + } - def stop() = logger.stop() + /** + * Parse the event logging information associated with the logs in the given directory. + */ + def parseLoggingInfo(logDir: String, fileSystem: FileSystem): EventLoggingInfo = { + parseLoggingInfo(new Path(logDir), fileSystem) + } } -// If compression is not enabled, compressionCodec is None -private[spark] case class EventLoggingInfo(logDir: String, compressionCodec: Option[String]) + +/** + * Information needed to process the event logs associated with an application. + */ +private[spark] case class EventLoggingInfo( + logPaths: Seq[Path], + sparkVersion: String, + compressionCodec: Option[CompressionCodec], + applicationComplete: Boolean = false) + +private[spark] object EventLoggingInfo { + def empty = EventLoggingInfo(Seq[Path](), "", None, applicationComplete = false) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 353a48661b0f7..76f3e327d60b8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -36,6 +36,22 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) private var queueFullErrorMessageLogged = false private var started = false + private val listenerThread = new Thread("SparkListenerBus") { + setDaemon(true) + override def run() { + while (true) { + val event = eventQueue.take + if (event == SparkListenerShutdown) { + // Get out of the while loop and shutdown the daemon thread + return + } + postToAll(event) + } + } + } + + // Exposed for testing + @volatile private[spark] var stopCalled = false /** * Start sending events to attached listeners. @@ -48,20 +64,8 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { if (started) { throw new IllegalStateException("Listener bus already started!") } + listenerThread.start() started = true - new Thread("SparkListenerBus") { - setDaemon(true) - override def run() { - while (true) { - val event = eventQueue.take - if (event == SparkListenerShutdown) { - // Get out of the while loop and shutdown the daemon thread - return - } - postToAll(event) - } - } - }.start() } def post(event: SparkListenerEvent) { @@ -93,9 +97,11 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { } def stop() { + stopCalled = true if (!started) { throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") } post(SparkListenerShutdown) + listenerThread.join() } } 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 db76178b65501..b03665fd56d33 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import java.io.InputStream -import java.net.URI import scala.io.Source @@ -26,63 +25,47 @@ import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.hadoop.fs.{Path, FileSystem} import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.Logging import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{JsonProtocol, Utils} +import org.apache.spark.util.JsonProtocol /** - * An EventBus that replays logged events from persisted storage + * A SparkListenerBus that replays logged events from persisted storage. + * + * This class expects files to be appropriately prefixed as specified in EventLoggingListener. + * There exists a one-to-one mapping between ReplayListenerBus and event logging applications. */ -private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus with Logging { - private val compressed = conf.getBoolean("spark.eventLog.compress", false) +private[spark] class ReplayListenerBus( + logPaths: Seq[Path], + fileSystem: FileSystem, + compressionCodec: Option[CompressionCodec]) + extends SparkListenerBus with Logging { - // Only used if compression is enabled - private lazy val compressionCodec = CompressionCodec.createCodec(conf) + private var replayed = false - /** - * Return a list of paths representing log files in the given directory. - */ - private def getLogFilePaths(logDir: String, fileSystem: FileSystem): Array[Path] = { - val path = new Path(logDir) - if (!fileSystem.exists(path) || !fileSystem.getFileStatus(path).isDir) { - logWarning("Log path provided is not a valid directory: %s".format(logDir)) - return Array[Path]() - } - val logStatus = fileSystem.listStatus(path) - if (logStatus == null || !logStatus.exists(!_.isDir)) { - logWarning("Log path provided contains no log files: %s".format(logDir)) - return Array[Path]() - } - logStatus.filter(!_.isDir).map(_.getPath).sortBy(_.getName) + if (logPaths.length == 0) { + logWarning("Log path provided contains no log files.") } /** * Replay each event in the order maintained in the given logs. + * This should only be called exactly once. */ - def replay(logDir: String): Boolean = { - val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) - val logPaths = getLogFilePaths(logDir, fileSystem) - if (logPaths.length == 0) { - return false - } - + def replay() { + assert(!replayed, "ReplayListenerBus cannot replay events more than once") logPaths.foreach { path => // Keep track of input streams at all levels to close them later // This is necessary because an exception can occur in between stream initializations var fileStream: Option[InputStream] = None var bufferedStream: Option[InputStream] = None var compressStream: Option[InputStream] = None - var currentLine = "" + var currentLine = "" try { - currentLine = "" fileStream = Some(fileSystem.open(path)) bufferedStream = Some(new FastBufferedInputStream(fileStream.get)) - compressStream = - if (compressed) { - Some(compressionCodec.compressedInputStream(bufferedStream.get)) - } else bufferedStream + compressStream = Some(wrapForCompression(bufferedStream.get)) - // Parse each line as an event and post it to all attached listeners + // Parse each line as an event and post the event to all attached listeners val lines = Source.fromInputStream(compressStream.get).getLines() lines.foreach { line => currentLine = line @@ -98,7 +81,11 @@ private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus compressStream.foreach(_.close()) } } - fileSystem.close() - true + replayed = true + } + + /** If a compression codec is specified, wrap the given stream in a compression stream. */ + private def wrapForCompression(stream: InputStream): InputStream = { + compressionCodec.map(_.compressedInputStream(stream)).getOrElse(stream) } } 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 ced20350d5356..378cf1aaebe7b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -75,6 +75,11 @@ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) @DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent +case class SparkListenerApplicationStart(appName: String, time: Long, sparkUser: String) + extends SparkListenerEvent + +case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent + /** An event used in the listener to shutdown the listener daemon thread. */ private[spark] case object SparkListenerShutdown extends SparkListenerEvent @@ -141,6 +146,16 @@ trait SparkListener { * Called when an RDD is manually unpersisted by the application */ def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { } + + /** + * Called when the application starts + */ + def onApplicationStart(applicationStart: SparkListenerApplicationStart) { } + + /** + * Called when the application ends + */ + def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { } } /** 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 729e120497571..d6df193d9bcf8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -61,6 +61,10 @@ private[spark] trait SparkListenerBus { sparkListeners.foreach(_.onBlockManagerRemoved(blockManagerRemoved)) case unpersistRDD: SparkListenerUnpersistRDD => sparkListeners.foreach(_.onUnpersistRDD(unpersistRDD)) + case applicationStart: SparkListenerApplicationStart => + sparkListeners.foreach(_.onApplicationStart(applicationStart)) + case applicationEnd: SparkListenerApplicationEnd => + sparkListeners.foreach(_.onApplicationEnd(applicationEnd)) case SparkListenerShutdown => } } 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 25b7472a99cdb..936e9db80573d 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 @@ -49,7 +49,7 @@ private[spark] class SparkDeploySchedulerBackend( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - sparkHome, sc.ui.appUIAddress, sc.eventLoggingInfo) + sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() diff --git a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala index 555486830a769..132502b75f8cd 100644 --- a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala +++ b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala @@ -23,6 +23,6 @@ import java.io.File * References a particular segment of a file (potentially the entire file), * based off an offset and a length. */ -private[spark] class FileSegment(val file: File, val offset: Long, val length : Long) { +private[spark] class FileSegment(val file: File, val offset: Long, val length: Long) { override def toString = "(name=%s, offset=%d, length=%d)".format(file.getName, offset, length) } diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index cd6610363e22c..f2633dfa8abd7 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -35,7 +35,7 @@ private[spark] class SparkUI( conf: SparkConf, val securityManager: SecurityManager, val listenerBus: SparkListenerBus, - val appName: String, + var appName: String, val basePath: String = "") extends WebUI(securityManager, basePath) with Logging { @@ -48,12 +48,17 @@ private[spark] class SparkUI( private val bindHost = Utils.localHostName() private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) - private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt + private val port = conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) // Maintain executor storage status through Spark events val storageStatusListener = new StorageStatusListener listenerBus.addListener(storageStatusListener) + /** Set the app name for this UI. */ + def setAppName(name: String) { + appName = name + } + /** Initialize all components of the server. */ def start() { attachTab(new JobProgressTab(this)) @@ -95,6 +100,6 @@ private[spark] class SparkUI( } private[spark] object SparkUI { - val DEFAULT_PORT = "4040" + val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" } diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index b5f2ec6831d26..0080a8b342b05 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -49,7 +49,7 @@ private[spark] class FileLogger( } private val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) - private var fileIndex = 0 + var fileIndex = 0 // Only used if compression is enabled private lazy val compressionCodec = CompressionCodec.createCodec(conf) @@ -57,10 +57,9 @@ private[spark] class FileLogger( // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None - private var writer: Option[PrintWriter] = { - createLogDir() - Some(createWriter()) - } + private var writer: Option[PrintWriter] = None + + createLogDir() /** * Create a logging directory with the given path. @@ -84,8 +83,8 @@ private[spark] class FileLogger( /** * Create a new writer for the file identified by the given path. */ - private def createWriter(): PrintWriter = { - val logPath = logDir + "/" + fileIndex + private def createWriter(fileName: String): PrintWriter = { + val logPath = logDir + "/" + fileName val uri = new URI(logPath) /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). @@ -147,13 +146,17 @@ private[spark] class FileLogger( } /** - * Start a writer for a new file if one does not already exit. + * Start a writer for a new file, closing the existing one if it exists. + * @param fileName Name of the new file, defaulting to the file index if not provided. */ - def start() { - writer.getOrElse { - fileIndex += 1 - writer = Some(createWriter()) + def newFile(fileName: String = "") { + fileIndex += 1 + writer.foreach(_.close()) + val name = fileName match { + case "" => fileIndex.toString + case _ => fileName } + writer = Some(createWriter(name)) } /** 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 19654892bf661..d990fd49ef834 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -62,6 +62,10 @@ private[spark] object JsonProtocol { blockManagerRemovedToJson(blockManagerRemoved) case unpersistRDD: SparkListenerUnpersistRDD => unpersistRDDToJson(unpersistRDD) + case applicationStart: SparkListenerApplicationStart => + applicationStartToJson(applicationStart) + case applicationEnd: SparkListenerApplicationEnd => + applicationEndToJson(applicationEnd) // Not used, but keeps compiler happy case SparkListenerShutdown => JNothing @@ -157,6 +161,18 @@ private[spark] object JsonProtocol { ("RDD ID" -> unpersistRDD.rddId) } + def applicationStartToJson(applicationStart: SparkListenerApplicationStart): JValue = { + ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ + ("App Name" -> applicationStart.appName) ~ + ("Timestamp" -> applicationStart.time) ~ + ("User" -> applicationStart.sparkUser) + } + + def applicationEndToJson(applicationEnd: SparkListenerApplicationEnd): JValue = { + ("Event" -> Utils.getFormattedClassName(applicationEnd)) ~ + ("Timestamp" -> applicationEnd.time) + } + /** ------------------------------------------------------------------- * * JSON serialization methods for classes SparkListenerEvents depend on | @@ -346,6 +362,8 @@ private[spark] object JsonProtocol { val blockManagerAdded = Utils.getFormattedClassName(SparkListenerBlockManagerAdded) val blockManagerRemoved = Utils.getFormattedClassName(SparkListenerBlockManagerRemoved) val unpersistRDD = Utils.getFormattedClassName(SparkListenerUnpersistRDD) + val applicationStart = Utils.getFormattedClassName(SparkListenerApplicationStart) + val applicationEnd = Utils.getFormattedClassName(SparkListenerApplicationEnd) (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) @@ -359,6 +377,8 @@ private[spark] object JsonProtocol { case `blockManagerAdded` => blockManagerAddedFromJson(json) case `blockManagerRemoved` => blockManagerRemovedFromJson(json) case `unpersistRDD` => unpersistRDDFromJson(json) + case `applicationStart` => applicationStartFromJson(json) + case `applicationEnd` => applicationEndFromJson(json) } } @@ -430,6 +450,17 @@ private[spark] object JsonProtocol { SparkListenerUnpersistRDD((json \ "RDD ID").extract[Int]) } + def applicationStartFromJson(json: JValue): SparkListenerApplicationStart = { + val appName = (json \ "App Name").extract[String] + val time = (json \ "Timestamp").extract[Long] + val sparkUser = (json \ "User").extract[String] + SparkListenerApplicationStart(appName, time, sparkUser) + } + + def applicationEndFromJson(json: JValue): SparkListenerApplicationEnd = { + SparkListenerApplicationEnd((json \ "Timestamp").extract[Long]) + } + /** --------------------------------------------------------------------- * * JSON deserialization methods for classes SparkListenerEvents depend on | 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 59da51f3e0297..166f48ce7342e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -26,7 +26,6 @@ import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.SortedSet import scala.io.Source import scala.reflect.ClassTag @@ -1022,4 +1021,11 @@ private[spark] object Utils extends Logging { def getHadoopFileSystem(path: URI): FileSystem = { FileSystem.get(path, SparkHadoopUtil.get.newConfiguration()) } + + /** + * Return a Hadoop FileSystem with the scheme encoded in the given path. + */ + def getHadoopFileSystem(path: String): FileSystem = { + getHadoopFileSystem(new URI(path)) + } } 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 7c843772bc2e0..dc704e07a81de 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import java.util.concurrent.Semaphore + import scala.collection.mutable import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} @@ -72,6 +74,49 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } } + test("bus.stop() waits for the event queue to completely drain") { + @volatile var drained = false + + // Tells the listener to stop blocking + val listenerWait = new Semaphore(1) + + // When stop has returned + val stopReturned = new Semaphore(1) + + class BlockingListener extends SparkListener { + override def onJobEnd(jobEnd: SparkListenerJobEnd) = { + listenerWait.acquire() + drained = true + } + } + + val bus = new LiveListenerBus + val blockingListener = new BlockingListener + + bus.addListener(blockingListener) + bus.start() + bus.post(SparkListenerJobEnd(0, JobSucceeded)) + + // the queue should not drain immediately + assert(!drained) + + new Thread("ListenerBusStopper") { + override def run() { + // stop() will block until notify() is called below + bus.stop() + stopReturned.release(1) + } + }.start() + + while (!bus.stopCalled) { + Thread.sleep(10) + } + + listenerWait.release() + stopReturned.acquire() + assert(drained) + } + test("basic creation of StageInfo") { val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) 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 beac656f573b4..8c06a2d9aa4ab 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui.jobs import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, Success} +import org.apache.spark.{LocalSparkContext, SparkConf, Success} import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} import org.apache.spark.scheduler._ import org.apache.spark.util.Utils 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 0342a8aff3c28..f75297a02dc8b 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import java.util.{Properties, UUID} +import java.util.Properties import scala.collection.Map @@ -52,6 +52,8 @@ class JsonProtocolSuite extends FunSuite { val blockManagerRemoved = SparkListenerBlockManagerRemoved( BlockManagerId("Scarce", "to be counted...", 100, 200)) val unpersistRdd = SparkListenerUnpersistRDD(12345) + val applicationStart = SparkListenerApplicationStart("The winner of all", 42L, "Garfield") + val applicationEnd = SparkListenerApplicationEnd(42L) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) @@ -64,6 +66,8 @@ class JsonProtocolSuite extends FunSuite { testEvent(blockManagerAdded, blockManagerAddedJsonString) testEvent(blockManagerRemoved, blockManagerRemovedJsonString) testEvent(unpersistRdd, unpersistRDDJsonString) + testEvent(applicationStart, applicationStartJsonString) + testEvent(applicationEnd, applicationEndJsonString) } test("Dependent Classes") { @@ -208,7 +212,13 @@ class JsonProtocolSuite extends FunSuite { case (e1: SparkListenerBlockManagerRemoved, e2: SparkListenerBlockManagerRemoved) => assertEquals(e1.blockManagerId, e2.blockManagerId) case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => - assert(e1.rddId === e2.rddId) + assert(e1.rddId == e2.rddId) + case (e1: SparkListenerApplicationStart, e2: SparkListenerApplicationStart) => + assert(e1.appName == e2.appName) + assert(e1.time == e2.time) + assert(e1.sparkUser == e2.sparkUser) + case (e1: SparkListenerApplicationEnd, e2: SparkListenerApplicationEnd) => + assert(e1.time == e2.time) case (SparkListenerShutdown, SparkListenerShutdown) => case _ => fail("Events don't match in types!") } @@ -553,4 +563,14 @@ class JsonProtocolSuite extends FunSuite { {"Event":"SparkListenerUnpersistRDD","RDD ID":12345} """ + private val applicationStartJsonString = + """ + {"Event":"SparkListenerApplicationStart","App Name":"The winner of all","Timestamp":42, + "User":"Garfield"} + """ + + private val applicationEndJsonString = + """ + {"Event":"SparkListenerApplicationEnd","Timestamp":42} + """ } diff --git a/docs/monitoring.md b/docs/monitoring.md index 15bfb041780da..4c91c3a5929bf 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -12,17 +12,77 @@ displays useful information about the application. This includes: * A list of scheduler stages and tasks * A summary of RDD sizes and memory usage -* Information about the running executors * Environmental information. +* Information about the running executors You can access this interface by simply opening `http://:4040` in a web browser. -If multiple SparkContexts are running on the same host, they will bind to succesive ports +If multiple SparkContexts are running on the same host, they will bind to successive ports beginning with 4040 (4041, 4042, etc). -Spark's Standalone Mode cluster manager also has its own -[web UI](spark-standalone.html#monitoring-and-logging). +Note that this information is only available for the duration of the application by default. +To view the web UI after the fact, set `spark.eventLog.enabled` to true before starting the +application. This configures Spark to log Spark events that encode the information displayed +in the UI to persisted storage. -Note that in both of these UIs, the tables are sortable by clicking their headers, +## Viewing After the Fact + +Spark's Standalone Mode cluster manager also has its own +[web UI](spark-standalone.html#monitoring-and-logging). If an application has logged events over +the course of its lifetime, then the Standalone master's web UI will automatically re-render the +application's UI after the application has finished. + +If Spark is run on Mesos or YARN, it is still possible to reconstruct the UI of a finished +application through Spark's history server, provided that the application's event logs exist. +You can start a the history server by executing: + + ./sbin/start-history-server.sh + +The base logging directory must be supplied, and should contain sub-directories that each +represents an application's event logs. This creates a web interface at +`http://:18080` by default. The history server depends on the following variables: + + + + + + + + + + + +
Environment VariableMeaning
SPARK_DAEMON_MEMORYMemory to allocate to the history server. (default: 512m).
SPARK_DAEMON_JAVA_OPTSJVM options for the history server (default: none).
+ +Further, the history server can be configured as follows: + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.history.updateInterval10 + The period, in seconds, at which information displayed by this history server is updated. + Each update checks for any changes made to the event logs in persisted storage. +
spark.history.retainedApplications250 + The number of application UIs to retain. If this cap is exceeded, then the oldest + applications will be removed. +
spark.history.ui.port18080 + The port to which the web interface of the history server binds. +
+ +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. # Metrics diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index 6b49244ba459d..bd96274021756 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -138,6 +138,6 @@ public static void main(String[] args) { System.out.print("Final w: "); printWeights(w); - System.exit(0); + sc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index 617e4a6d045e0..2a4278d3c30e5 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -126,6 +126,6 @@ public Stats call(Stats stats, Stats stats2) { for (Tuple2 t : output) { System.out.println(t._1() + "\t" + t._2()); } - System.exit(0); + jsc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index 8513ba07e7705..e31f676f5fd4c 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -117,6 +117,6 @@ public Double call(Double sum) { System.out.println(tuple._1() + " has rank: " + tuple._2() + "."); } - System.exit(0); + ctx.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 6cfe25c80ecc6..1d776940f06c6 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -96,6 +96,6 @@ public Tuple2 call(Tuple2 e) { } while (nextCount != oldCount); System.out.println("TC has " + tc.count() + " edges."); - System.exit(0); + sc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index 3ae1d8f7ca938..87c1b80981961 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -48,14 +48,14 @@ public Iterable call(String s) { return Arrays.asList(SPACE.split(s)); } }); - + JavaPairRDD ones = words.mapToPair(new PairFunction() { @Override public Tuple2 call(String s) { return new Tuple2(s, 1); } }); - + JavaPairRDD counts = ones.reduceByKey(new Function2() { @Override public Integer call(Integer i1, Integer i2) { @@ -67,6 +67,6 @@ public Integer call(Integer i1, Integer i2) { for (Tuple2 tuple : output) { System.out.println(tuple._1() + ": " + tuple._2()); } - System.exit(0); + ctx.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java index 64a3a04fb7296..c516199d61c72 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java @@ -85,6 +85,6 @@ public static void main(String[] args) { outputDir + "/productFeatures"); System.out.println("Final user/product features written to " + outputDir); - System.exit(0); + sc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index 7b0ec36424e97..7461609ab9e8f 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -79,6 +79,6 @@ public static void main(String[] args) { double cost = model.computeCost(points.rdd()); System.out.println("Cost: " + cost); - System.exit(0); + sc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index cd8879ff886e2..e3ab87cc722f3 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -77,6 +77,6 @@ public static void main(String[] args) { System.out.print("Final w: " + model.weights()); - System.exit(0); + sc.stop(); } } 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 4d2f45df85fc6..c8c916bb45e00 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -56,6 +56,6 @@ object BroadcastTest { println("Iteration %d took %.0f milliseconds".format(i, (System.nanoTime - startTime) / 1E6)) } - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala index ee283ce6abac2..1f8d7cb5995b8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala @@ -58,7 +58,7 @@ import org.apache.spark.SparkContext._ prod_id, quantity) VALUES ('charlie', 1385983649000, 'iphone', 2); */ - + /** * This example demonstrates how to read and write to cassandra column family created using CQL3 * using Spark. diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index fdb976dfc6aba..be7d39549a28d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -34,6 +34,6 @@ object ExceptionHandlingTest { } } - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 36534e59353cd..29114c6dabcdb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -28,7 +28,7 @@ object GroupByTest { "Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers]") System.exit(1) } - + var numMappers = if (args.length > 1) args(1).toInt else 2 var numKVPairs = if (args.length > 2) args(2).toInt else 1000 var valSize = if (args.length > 3) args(3).toInt else 1000 @@ -52,7 +52,6 @@ object GroupByTest { println(pairs1.groupByKey(numReducers).count) - System.exit(0) + sc.stop() } } - 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 65d67356be2f6..700121d16dd60 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -30,7 +30,7 @@ object HBaseTest { val conf = HBaseConfiguration.create() - // Other options for configuring scan behavior are available. More information available at + // Other options for configuring scan behavior are available. More information available at // http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html conf.set(TableInputFormat.INPUT_TABLE, args(1)) @@ -41,12 +41,12 @@ object HBaseTest { admin.createTable(tableDesc) } - val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], + val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], classOf[org.apache.hadoop.hbase.client.Result]) hBaseRDD.count() - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index c3597d94a224e..dd6d5205133be 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -32,6 +32,6 @@ object HdfsTest { val end = System.currentTimeMillis() println("Iteration " + iter + " took " + (end-start) + " ms") } - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index 0095cb8425456..37ad4bd0999bd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -120,7 +120,7 @@ object LocalALS { } } printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) - + val R = generateR() // Initialize m and u randomly diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index 4aef04fc060b6..97321ab8f41db 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -51,6 +51,6 @@ object MultiBroadcastTest { // Collect the small RDD so we can print the observed sizes locally. observedSizes.collect().foreach(i => println(i)) - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 1fdb324b89f3a..d05eedd31caa0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -27,7 +27,7 @@ object SimpleSkewedGroupByTest { System.err.println("Usage: SimpleSkewedGroupByTest " + "[numMappers] [numKVPairs] [valSize] [numReducers] [ratio]") System.exit(1) - } + } var numMappers = if (args.length > 1) args(1).toInt else 2 var numKVPairs = if (args.length > 2) args(2).toInt else 1000 @@ -58,14 +58,13 @@ object SimpleSkewedGroupByTest { }.cache // Enforce that everything has been calculated and in cache pairs1.count - + println("RESULT: " + pairs1.groupByKey(numReducers).count) // Print how many keys each reducer got (for debugging) // println("RESULT: " + pairs1.groupByKey(numReducers) // .map{case (k,v) => (k, v.size)} // .collectAsMap) - System.exit(0) + sc.stop() } } - diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 966478fe4a258..fd9f043247d18 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -27,7 +27,7 @@ object SkewedGroupByTest { System.err.println( "Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers]") System.exit(1) - } + } var numMappers = if (args.length > 1) args(1).toInt else 2 var numKVPairs = if (args.length > 2) args(2).toInt else 1000 @@ -53,10 +53,9 @@ object SkewedGroupByTest { }.cache() // Enforce that everything has been calculated and in cache pairs1.count() - + println(pairs1.groupByKey(numReducers).count()) - System.exit(0) + sc.stop() } } - diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index f59ab7e7cc24a..68f151a2c47fe 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -112,7 +112,7 @@ object SparkALS { val sc = new SparkContext(host, "SparkALS", System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) - + val R = generateR() // Initialize m and u randomly @@ -137,6 +137,6 @@ object SparkALS { println() } - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index e698b9bf376e1..d8de8745c15d9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -52,7 +52,7 @@ object SparkHdfsLR { val inputPath = args(1) val conf = SparkHadoopUtil.get.newConfiguration() val sc = new SparkContext(args(0), "SparkHdfsLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) )) @@ -73,6 +73,6 @@ object SparkHdfsLR { } println("Final w: " + w) - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index 9fe24652358f3..1a8b21618e23a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -28,16 +28,16 @@ import org.apache.spark.SparkContext._ object SparkKMeans { val R = 1000 // Scaling factor val rand = new Random(42) - + def parseVector(line: String): Vector = { new Vector(line.split(' ').map(_.toDouble)) } - + def closestPoint(p: Vector, centers: Array[Vector]): Int = { var index = 0 var bestIndex = 0 var closest = Double.PositiveInfinity - + for (i <- 0 until centers.length) { val tempDist = p.squaredDist(centers(i)) if (tempDist < closest) { @@ -45,7 +45,7 @@ object SparkKMeans { bestIndex = i } } - + bestIndex } @@ -60,22 +60,22 @@ object SparkKMeans { val data = lines.map(parseVector _).cache() val K = args(2).toInt val convergeDist = args(3).toDouble - + val kPoints = data.takeSample(withReplacement = false, K, 42).toArray var tempDist = 1.0 while(tempDist > convergeDist) { val closest = data.map (p => (closestPoint(p, kPoints), (p, 1))) - + val pointStats = closest.reduceByKey{case ((x1, y1), (x2, y2)) => (x1 + x2, y1 + y2)} - + val newPoints = pointStats.map {pair => (pair._1, pair._2._1 / pair._2._2)}.collectAsMap() - + tempDist = 0.0 for (i <- 0 until K) { tempDist += kPoints(i).squaredDist(newPoints(i)) } - + for (newP <- newPoints) { kPoints(newP._1) = newP._2 } @@ -84,6 +84,6 @@ object SparkKMeans { println("Final centers:") kPoints.foreach(println) - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index c54a55bdb4a11..3a2699d4d996b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -66,6 +66,6 @@ object SparkLR { } println("Final w: " + w) - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index d203f4d20e15f..45b6e10f3ea9e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -57,7 +57,6 @@ object SparkPageRank { val output = ranks.collect() output.foreach(tup => println(tup._1 + " has rank: " + tup._2 + ".")) - System.exit(0) + ctx.stop() } } - diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 24e8afa26bc5f..eb47cf027cb10 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -70,6 +70,6 @@ object SparkTC { } while (nextCount != oldCount) println("TC has " + tc.count() + " edges.") - System.exit(0) + spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala index 53b303d658386..5698d4746495d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -75,6 +75,6 @@ object SparkTachyonHdfsLR { } println("Final w: " + w) - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala index ce78f0876ed7c..2b207fd8d3e16 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -34,10 +34,10 @@ object SparkTachyonPi { } val spark = new SparkContext(args(0), "SparkTachyonPi", System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) - + val slices = if (args.length > 1) args(1).toInt else 2 val n = 100000 * slices - + val rdd = spark.parallelize(1 to n, slices) rdd.persist(StorageLevel.OFF_HEAP) val count = rdd.map { i => @@ -46,7 +46,7 @@ object SparkTachyonPi { if (x * x + y * y < 1) 1 else 0 }.reduce(_ + _) println("Pi is roughly " + 4.0 * count / n) - + spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index 7aac6a13597e6..dee3cb6c0abae 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -79,7 +79,7 @@ object WikipediaPageRankStandalone { val time = (System.currentTimeMillis - startTime) / 1000.0 println("Completed %d iterations in %f seconds: %f seconds per iteration" .format(numIterations, time, time / numIterations)) - System.exit(0) + sc.stop() } def parseArticle(line: String): (String, Array[String]) = { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala index a177435e606ab..61b9655cd3759 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.examples.mllib - + import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.linalg.Vectors diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala index 49d09692c8e4a..9aeebf58eabfb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.examples.mllib - + import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.linalg.Vectors diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index 954bcc9b6ef5d..1c0ce3111e290 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -53,4 +53,3 @@ object HdfsWordCount { ssc.awaitTermination() } } - diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 6bccd1d88401a..cca0be2cbb9c9 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -61,7 +61,7 @@ object KafkaWordCount { val wordCounts = words.map(x => (x, 1L)) .reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2) wordCounts.print() - + ssc.start() ssc.awaitTermination() } @@ -83,7 +83,7 @@ object KafkaWordCountProducer { val props = new Properties() props.put("metadata.broker.list", brokers) props.put("serializer.class", "kafka.serializer.StringEncoder") - + val config = new ProducerConfig(props) val producer = new Producer[String, String](config) @@ -102,4 +102,3 @@ object KafkaWordCountProducer { } } - diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index 0a68ac84c2424..656222e0c1b31 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -26,7 +26,7 @@ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.mqtt._ /** - * A simple Mqtt publisher for demonstration purposes, repeatedly publishes + * A simple Mqtt publisher for demonstration purposes, repeatedly publishes * Space separated String Message "hello mqtt demo for spark streaming" */ object MQTTPublisher { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala index 4d4968ba6ae3e..612ecf7b7821a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala @@ -24,7 +24,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ object QueueStream { - + def main(args: Array[String]) { if (args.length < 1) { System.err.println("Usage: QueueStream ") @@ -37,23 +37,22 @@ object QueueStream { val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) - // Create the queue through which RDDs can be pushed to + // Create the queue through which RDDs can be pushed to // a QueueInputDStream val rddQueue = new SynchronizedQueue[RDD[Int]]() - + // Create the QueueInputDStream and use it do some processing val inputStream = ssc.queueStream(rddQueue) val mappedStream = inputStream.map(x => (x % 10, 1)) val reducedStream = mappedStream.reduceByKey(_ + _) - reducedStream.print() + reducedStream.print() ssc.start() - + // Create and push some RDDs into for (i <- 1 to 30) { rddQueue += ssc.sparkContext.makeRDD(1 to 1000, 10) Thread.sleep(1000) } ssc.stop() - System.exit(0) } } diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index c2d84a8e0861e..14f65a2f8d46c 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -58,7 +58,7 @@ object StatefulNetworkWordCount { ssc.checkpoint(".") // Create a NetworkInputDStream on target ip:port and count the - // words in input stream of \n delimited test (eg. generated by 'nc') + // words in input stream of \n delimited test (eg. generated by 'nc') val lines = ssc.socketTextStream(args(1), args(2).toInt) val words = lines.flatMap(_.split(" ")) val wordDstream = words.map(x => (x, 1)) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 35f8f885f8f0e..445d2028582af 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -60,7 +60,7 @@ object SimpleZeroMQPublisher { * To work with zeroMQ, some native libraries have to be installed. * Install zeroMQ (release 2.1) core libraries. [ZeroMQ Install guide] * (http://www.zeromq.org/intro:get-the-software) - * + * * Usage: ZeroMQWordCount * In local mode, should be 'local[n]' with n > 1 * and describe where zeroMq publisher is running. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index ae27c57799873..a6c049e517ee0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -30,7 +30,6 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: - * * The Java stubs necessary for the Python mllib bindings. */ @DeveloperApi diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 5a45f12f1aa12..18658850a2f64 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -29,7 +29,6 @@ import org.apache.spark.rdd.RDD /** * :: Experimental :: - * * Model for Naive Bayes Classifiers. * * @param labels list of labels diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 8f565eb60a60f..90cf8525df523 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -78,7 +78,6 @@ class KMeans private ( /** * :: Experimental :: - * * Set the number of runs of the algorithm to execute in parallel. We initialize the algorithm * this many times with random starting conditions (configured by the initialization mode), then * return the best clustering found over any run. Default: 1. @@ -398,9 +397,6 @@ object KMeans { MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm) } - /** - * :: Experimental :: - */ @Experimental def main(args: Array[String]) { if (args.length < 4) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 89d5c03d76c42..56b8fdcda66eb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -34,7 +34,6 @@ case class MatrixEntry(i: Long, j: Long, value: Double) /** * :: Experimental :: - * * Represents a matrix in coordinate format. * * @param entries matrix entries diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala index 24c123ab7eb51..132b3af72d9ce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -26,7 +26,6 @@ import org.apache.spark.mllib.linalg.SingularValueDecomposition /** * :: Experimental :: - * * Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. */ @Experimental @@ -34,7 +33,6 @@ case class IndexedRow(index: Long, vector: Vector) /** * :: Experimental :: - * * Represents a row-oriented [[org.apache.spark.mllib.linalg.distributed.DistributedMatrix]] with * indexed rows. * diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 8d32c1a6dbba1..f65f43dd3007b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -30,7 +30,6 @@ import org.apache.spark.Logging /** * :: Experimental :: - * * Represents a row-oriented distributed Matrix with no meaningful row indices. * * @param rows rows stored as an RDD[Vector] diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 1176dc9dbc08d..679842f831c2a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * :: DeveloperApi :: - * * Class used to compute the gradient for a loss function, given a single data point. */ @DeveloperApi @@ -56,7 +55,6 @@ abstract class Gradient extends Serializable { /** * :: DeveloperApi :: - * * Compute gradient and loss for a logistic loss function, as used in binary classification. * See also the documentation for the precise formulation. */ @@ -100,7 +98,6 @@ class LogisticGradient extends Gradient { /** * :: DeveloperApi :: - * * Compute gradient and loss for a Least-squared loss function, as used in linear regression. * This is correct for the averaged least squares loss function (mean squared error) * L = 1/n ||A weights-y||^2 @@ -135,7 +132,6 @@ class LeastSquaresGradient extends Gradient { /** * :: DeveloperApi :: - * * Compute gradient and loss for a Hinge loss function, as used in SVM binary classification. * See also the documentation for the precise formulation. * NOTE: This assumes that the labels are {0,1} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 04267d967dcad..f60417f21d4b9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -28,7 +28,6 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * :: DeveloperApi :: - * * Class used to solve an optimization problem using Gradient Descent. * @param gradient Gradient function to be used. * @param updater Updater to be used to update weights after every iteration. @@ -113,7 +112,6 @@ class GradientDescent(private var gradient: Gradient, private var updater: Updat /** * :: DeveloperApi :: - * * Top-level method to run gradient descent. */ @DeveloperApi diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index 0a313f3104b14..e41d9bbe18c37 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.linalg.Vector /** * :: DeveloperApi :: - * * Trait for optimization problem solvers. */ @DeveloperApi diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index e67816796c6b1..3ed3a5b9b3843 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -26,7 +26,6 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * :: DeveloperApi :: - * * Class used to perform steps (weight update) using Gradient Descent methods. * * For general minimization problems, or for regularized problems of the form @@ -64,7 +63,6 @@ abstract class Updater extends Serializable { /** * :: DeveloperApi :: - * * A simple updater for gradient descent *without* any regularization. * Uses a step-size decreasing with the square root of the number of iterations. */ @@ -86,7 +84,6 @@ class SimpleUpdater extends Updater { /** * :: DeveloperApi :: - * * Updater for L1 regularized problems. * R(w) = ||w||_1 * Uses a step-size decreasing with the square root of the number of iterations. @@ -131,7 +128,6 @@ class L1Updater extends Updater { /** * :: DeveloperApi :: - * * Updater for L2 regularized problems. * R(w) = 1/2 ||w||^2 * Uses a step-size decreasing with the square root of the number of iterations. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 60cbb1c1e1d86..5cc47de8ffdfc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -140,7 +140,6 @@ class ALS private ( /** * :: Experimental :: - * * Sets the constant used in computing confidence in implicit ALS. Default: 1.0. */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index e05224fc7caf2..471546cd82c7d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -70,7 +70,6 @@ class MatrixFactorizationModel( /** * :: DeveloperApi :: - * * Predict the rating of many users for many products. * This is a Java stub for python predictAll() * diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index c24f5afb99686..3bd0017aa196a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -103,7 +103,6 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] /** * :: Experimental :: - * * Set if the algorithm should validate data before training. Default true. */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index c8a966cd5f5a8..3019447ce4cd9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -35,7 +35,6 @@ import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * :: Experimental :: - * * A class that implements a decision tree algorithm for classification and regression. It * supports both continuous and categorical features. * @param strategy The configuration parameters for the tree algorithm which specify the type diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala index 017f84f3b9e8b..79a01f58319e8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.Experimental /** * :: Experimental :: - * * Enum to select the algorithm for the decision tree */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala index c0254c32c2dce..f4c877232750f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.Experimental /** * :: Experimental :: - * * Enum to describe whether a feature is "continuous" or "categorical" */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala index b3e8b224beeaa..7da976e55a722 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.Experimental /** * :: Experimental :: - * * Enum for selecting the quantile calculation strategy */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 482faaa9e7256..8767aca47cd5a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ /** * :: Experimental :: - * * Stores all the configuration options for tree construction * @param algo classification or regression * @param impurity criterion used for information gain calculation diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 55c43f2fcf9c5..60f43e9278d2a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: - * * Class for calculating [[http://en.wikipedia.org/wiki/Binary_entropy_function entropy]] during * binary classification. */ @@ -32,7 +31,6 @@ object Entropy extends Impurity { /** * :: DeveloperApi :: - * * entropy calculation * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index c923b8e8f4cf1..c51d76d9b4c5b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: - * * Class for calculating the * [[http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity Gini impurity]] * during binary classification. @@ -31,7 +30,6 @@ object Gini extends Impurity { /** * :: DeveloperApi :: - * * Gini coefficient calculation * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index f407796596c6c..8eab247cf0932 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: - * * Trait for calculating information gain. */ @Experimental @@ -29,7 +28,6 @@ trait Impurity extends Serializable { /** * :: DeveloperApi :: - * * information calculation for binary classification * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 @@ -40,7 +38,6 @@ trait Impurity extends Serializable { /** * :: DeveloperApi :: - * * information calculation for regression * @param count number of instances * @param sum sum of labels diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index 2c64644f4ed0f..47d07122af30f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: - * * Class for calculating variance during regression */ @Experimental @@ -31,7 +30,6 @@ object Variance extends Impurity { /** * :: DeveloperApi :: - * * variance calculation * @param count number of instances * @param sum sum of labels diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 0f76f4a049057..bf692ca8c4bd7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.linalg.Vector /** * :: Experimental :: - * * Model to store the decision tree parameters * @param topNode root node * @param algo algorithm type -- classification or regression diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index d36b58e92ced6..cc8a24cce9614 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: - * * Information gain statistics for each split * @param gain information gain value * @param impurity current node impurity diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 339972141498c..682f213f411a7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.linalg.Vector /** * :: DeveloperApi :: - * * Node in a decision tree * @param id integer node id * @param predict predicted value at the node diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index 8bbb343079b49..d7ffd386c05ee 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -22,7 +22,6 @@ import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType /** * :: DeveloperApi :: - * * Split applied to a feature * @param feature feature index * @param threshold threshold for continuous feature diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index 230c409e1be33..45f95482a1def 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.regression.LabeledPoint /** * :: DeveloperApi :: - * * A collection of methods used to validate data before applying ML algorithms. */ @DeveloperApi diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala index e693d13703987..6eaebaf7dba9f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala @@ -25,7 +25,6 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: - * * Generate test data for KMeans. This class first chooses k cluster centers * from a d-dimensional Gaussian distribution scaled by factor r and then creates a Gaussian * cluster with scale 1 around each center. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 140ff92869176..c8e160d00c2d6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -30,7 +30,6 @@ import org.apache.spark.mllib.regression.LabeledPoint /** * :: DeveloperApi :: - * * Generate sample data used for Linear Data. This class generates * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the * response variable `Y`. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index ca06b9ad58538..c82cd8fd4641c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -27,7 +27,6 @@ import org.apache.spark.mllib.linalg.Vectors /** * :: DeveloperApi :: - * * Generate test data for LogisticRegression. This class chooses positive labels * with probability `probOne` and scales features for positive examples by `eps`. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 3bd86d6813375..3f413faca6bb4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -27,29 +27,28 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: + * Generate RDD(s) containing data for Matrix Factorization. * -* Generate RDD(s) containing data for Matrix Factorization. -* -* This method samples training entries according to the oversampling factor -* 'trainSampFact', which is a multiplicative factor of the number of -* degrees of freedom of the matrix: rank*(m+n-rank). -* -* It optionally samples entries for a testing matrix using -* 'testSampFact', the percentage of the number of training entries -* to use for testing. -* -* This method takes the following inputs: -* sparkMaster (String) The master URL. -* outputPath (String) Directory to save output. -* m (Int) Number of rows in data matrix. -* n (Int) Number of columns in data matrix. -* rank (Int) Underlying rank of data matrix. -* trainSampFact (Double) Oversampling factor. -* noise (Boolean) Whether to add gaussian noise to training data. -* sigma (Double) Standard deviation of added gaussian noise. -* test (Boolean) Whether to create testing RDD. -* testSampFact (Double) Percentage of training data to use as test data. -*/ + * This method samples training entries according to the oversampling factor + * 'trainSampFact', which is a multiplicative factor of the number of + * degrees of freedom of the matrix: rank*(m+n-rank). + * + * It optionally samples entries for a testing matrix using + * 'testSampFact', the percentage of the number of training entries + * to use for testing. + * + * This method takes the following inputs: + * sparkMaster (String) The master URL. + * outputPath (String) Directory to save output. + * m (Int) Number of rows in data matrix. + * n (Int) Number of columns in data matrix. + * rank (Int) Underlying rank of data matrix. + * trainSampFact (Double) Oversampling factor. + * noise (Boolean) Whether to add gaussian noise to training data. + * sigma (Double) Standard deviation of added gaussian noise. + * test (Boolean) Whether to create testing RDD. + * testSampFact (Double) Percentage of training data to use as test data. + */ @DeveloperApi object MFDataGenerator { def main(args: Array[String]) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 7f9804deaf33f..ac2360c429e2b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -124,7 +124,6 @@ object MLUtils { /** * :: Experimental :: - * * Load labeled data from a file. The data format used here is * , ... * where , are feature values in Double and is the corresponding label as Double. @@ -146,7 +145,6 @@ object MLUtils { /** * :: Experimental :: - * * Save labeled data to a file. The data format used here is * , ... * where , are feature values in Double and is the corresponding label as Double. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index 87a6f2a0c3976..ba8190b0e07e8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -29,7 +29,6 @@ import org.apache.spark.mllib.regression.LabeledPoint /** * :: DeveloperApi :: - * * Generate sample data used for SVM. This class generates uniform random values * for the features and adds Gaussian noise with weight 0.1 to generate labels. */ diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index 20a0e309d1494..7ef251d24c77e 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -15,8 +15,9 @@ # limitations under the License. # -from numpy import ndarray, copyto, float64, int64, int32, ones, array_equal, array, dot, shape +from numpy import ndarray, copyto, float64, int64, int32, ones, array_equal, array, dot, shape, complex, issubdtype from pyspark import SparkContext, RDD +import numpy as np from pyspark.serializers import Serializer import struct @@ -47,13 +48,22 @@ def _deserialize_byte_array(shape, ba, offset): return ar.copy() def _serialize_double_vector(v): - """Serialize a double vector into a mutually understood format.""" + """Serialize a double vector into a mutually understood format. + + >>> x = array([1,2,3]) + >>> y = _deserialize_double_vector(_serialize_double_vector(x)) + >>> array_equal(y, array([1.0, 2.0, 3.0])) + True + """ if type(v) != ndarray: raise TypeError("_serialize_double_vector called on a %s; " "wanted ndarray" % type(v)) + """complex is only datatype that can't be converted to float64""" + if issubdtype(v.dtype, complex): + raise TypeError("_serialize_double_vector called on a %s; " + "wanted ndarray" % type(v)) if v.dtype != float64: - raise TypeError("_serialize_double_vector called on an ndarray of %s; " - "wanted ndarray of float64" % v.dtype) + v = v.astype(float64) if v.ndim != 1: raise TypeError("_serialize_double_vector called on a %ddarray; " "wanted a 1darray" % v.ndim) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index 3ebf288130fb6..910b31d209e13 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -116,14 +116,14 @@ trait SparkILoopInit { } } - def initializeSpark() { + def initializeSpark() { intp.beQuietDuring { command(""" @transient val sc = org.apache.spark.repl.Main.interp.createSparkContext(); """) command("import org.apache.spark.SparkContext._") } - echo("Spark context available as sc.") + echo("Spark context available as sc.") } // code to be executed only after the interpreter is initialized diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh new file mode 100755 index 0000000000000..4a90c68763b68 --- /dev/null +++ b/sbin/start-history-server.sh @@ -0,0 +1,37 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Starts the history server on the machine this script is executed on. +# +# Usage: start-history-server.sh [] +# Example: ./start-history-server.sh --dir /tmp/spark-events --port 18080 +# + +sbin=`dirname "$0"` +sbin=`cd "$sbin"; pwd` + +if [ $# -lt 1 ]; then + echo "Usage: ./start-history-server.sh " + echo "Example: ./start-history-server.sh /tmp/spark-events" + exit +fi + +LOG_DIR=$1 + +"$sbin"/spark-daemon.sh start org.apache.spark.deploy.history.HistoryServer 1 --dir "$LOG_DIR" diff --git a/sbin/stop-history-server.sh b/sbin/stop-history-server.sh new file mode 100755 index 0000000000000..c0034ad641cbe --- /dev/null +++ b/sbin/stop-history-server.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Stops the history server on the machine this script is executed on. + +sbin=`dirname "$0"` +sbin=`cd "$sbin"; pwd` + +"$sbin"/spark-daemon.sh stop org.apache.spark.deploy.history.HistoryServer 1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 505ad0a2c77c1..4d7c86a3a4fc7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -82,30 +82,13 @@ private[sql] case class ParquetRelation(val path: String) private[sql] object ParquetRelation { def enableLogForwarding() { - // Note: Parquet does not use forwarding to parent loggers which - // is required for the JUL-SLF4J bridge to work. Also there is - // a default logger that appends to Console which needs to be - // reset. - import org.slf4j.bridge.SLF4JBridgeHandler - import java.util.logging.Logger - import java.util.logging.LogManager - - val loggerNames = Seq( - "parquet.hadoop.ColumnChunkPageWriteStore", - "parquet.hadoop.InternalParquetRecordWriter", - "parquet.hadoop.ParquetRecordReader", - "parquet.hadoop.ParquetInputFormat", - "parquet.hadoop.ParquetOutputFormat", - "parquet.hadoop.ParquetFileReader", - "parquet.hadoop.InternalParquetRecordReader", - "parquet.hadoop.codec.CodecConfig") - LogManager.getLogManager.reset() - SLF4JBridgeHandler.install() - for(name <- loggerNames) { - val logger = Logger.getLogger(name) - logger.setParent(Logger.getLogger(Logger.GLOBAL_LOGGER_NAME)) - logger.setUseParentHandlers(true) - } + // Note: Logger.getLogger("parquet") has a default logger + // that appends to Console which needs to be cleared. + val parquetLogger = java.util.logging.Logger.getLogger("parquet") + parquetLogger.getHandlers.foreach(parquetLogger.removeHandler) + // TODO(witgo): Need to set the log level ? + // if(parquetLogger.getLevel != null) parquetLogger.setLevel(null) + if (!parquetLogger.getUseParentHandlers) parquetLogger.setUseParentHandlers(true) } // The element type for the RDDs that this relation maps to. diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala index a4e42189f3309..8f6e3ea9dce40 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala @@ -25,7 +25,7 @@ import scala.util.Random class UISuite extends FunSuite with BeforeAndAfterAll { - ignore("Testing") { + test("Testing") { runStreaming(1000000) }