diff --git a/assembly/pom.xml b/assembly/pom.xml
index c65192bde64c6..4e2b773e7d2f3 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../pom.xml
diff --git a/bagel/pom.xml b/bagel/pom.xml
index 93db0d5efda5f..0327ffa402671 100644
--- a/bagel/pom.xml
+++ b/bagel/pom.xml
@@ -21,7 +21,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../pom.xml
diff --git a/bin/spark-submit b/bin/spark-submit
index c557311b4b20e..f92d90c3a66b0 100755
--- a/bin/spark-submit
+++ b/bin/spark-submit
@@ -22,6 +22,9 @@
export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)"
ORIG_ARGS=("$@")
+# Set COLUMNS for progress bar
+export COLUMNS=`tput cols`
+
while (($#)); do
if [ "$1" = "--deploy-mode" ]; then
SPARK_SUBMIT_DEPLOY_MODE=$2
diff --git a/core/pom.xml b/core/pom.xml
index 492eddda744c2..1feb00b3a7fb8 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -21,7 +21,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../pom.xml
diff --git a/core/src/main/java/org/apache/spark/SparkStageInfo.java b/core/src/main/java/org/apache/spark/SparkStageInfo.java
index 04e2247210ecc..fd74321093658 100644
--- a/core/src/main/java/org/apache/spark/SparkStageInfo.java
+++ b/core/src/main/java/org/apache/spark/SparkStageInfo.java
@@ -26,6 +26,7 @@
public interface SparkStageInfo {
int stageId();
int currentAttemptId();
+ long submissionTime();
String name();
int numTasks();
int numActiveTasks();
diff --git a/core/src/main/java/org/apache/spark/api/java/function/package.scala b/core/src/main/java/org/apache/spark/api/java/function/package.scala
index 7f91de653a64a..0f9bac7164162 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/package.scala
+++ b/core/src/main/java/org/apache/spark/api/java/function/package.scala
@@ -22,4 +22,4 @@ package org.apache.spark.api.java
* these interfaces to pass functions to various Java API methods for Spark. Please visit Spark's
* Java programming guide for more details.
*/
-package object function
\ No newline at end of file
+package object function
diff --git a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js
index badd85ed48c82..d33c5c769d683 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js
+++ b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js
@@ -26,13 +26,6 @@ $(function() {
// Switch the class of the arrow from open to closed.
$(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-open');
$(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-closed');
-
- // If clicking caused the metrics to expand, automatically check all options for additional
- // metrics (don't trigger a click when collapsing metrics, because it leads to weird
- // toggling behavior).
- if (!$(additionalMetricsDiv).hasClass('collapsed')) {
- $(this).parent().find('input:checkbox:not(:checked)').trigger('click');
- }
});
$("input:checkbox:not(:checked)").each(function() {
@@ -48,6 +41,16 @@ $(function() {
stripeTables();
});
+ $("#select-all-metrics").click(function() {
+ if (this.checked) {
+ // Toggle all un-checked options.
+ $('input:checkbox:not(:checked)').trigger('click');
+ } else {
+ // Toggle all checked options.
+ $('input:checkbox:checked').trigger('click');
+ }
+ });
+
// Trigger a click on the checkbox if a user clicks the label next to it.
$("span.additional-metric-title").click(function() {
$(this).parent().find('input:checkbox').trigger('click');
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 6b8c4eb9afe32..e95819d75a4c5 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -50,7 +50,7 @@ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkD
import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
import org.apache.spark.scheduler.local.LocalBackend
import org.apache.spark.storage._
-import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.{SparkUI, ConsoleProgressBar}
import org.apache.spark.ui.jobs.JobProgressListener
import org.apache.spark.util._
@@ -58,12 +58,26 @@ import org.apache.spark.util._
* Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
* cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster.
*
+ * Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before
+ * creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details.
+ *
* @param config a Spark Config object describing the application configuration. Any settings in
* this config overrides the default configs as well as system properties.
*/
-
class SparkContext(config: SparkConf) extends Logging {
+ // The call site where this SparkContext was constructed.
+ private val creationSite: CallSite = Utils.getCallSite()
+
+ // If true, log warnings instead of throwing exceptions when multiple SparkContexts are active
+ private val allowMultipleContexts: Boolean =
+ config.getBoolean("spark.driver.allowMultipleContexts", false)
+
+ // In order to prevent multiple SparkContexts from being active at the same time, mark this
+ // context as having started construction.
+ // NOTE: this must be placed at the beginning of the SparkContext constructor.
+ SparkContext.markPartiallyConstructed(this, allowMultipleContexts)
+
// This is used only by YARN for now, but should be relevant to other cluster types (Mesos,
// etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It
// contains a map from hostname to a list of input format splits on the host.
@@ -233,6 +247,13 @@ class SparkContext(config: SparkConf) extends Logging {
val statusTracker = new SparkStatusTracker(this)
+ private[spark] val progressBar: Option[ConsoleProgressBar] =
+ if (conf.getBoolean("spark.ui.showConsoleProgress", true) && !log.isInfoEnabled) {
+ Some(new ConsoleProgressBar(this))
+ } else {
+ None
+ }
+
// Initialize the Spark UI
private[spark] val ui: Option[SparkUI] =
if (conf.getBoolean("spark.ui.enabled", true)) {
@@ -1166,27 +1187,30 @@ class SparkContext(config: SparkConf) extends Logging {
/** Shut down the SparkContext. */
def stop() {
- postApplicationEnd()
- ui.foreach(_.stop())
- // Do this only if not stopped already - best case effort.
- // prevent NPE if stopped more than once.
- val dagSchedulerCopy = dagScheduler
- dagScheduler = null
- if (dagSchedulerCopy != null) {
- env.metricsSystem.report()
- metadataCleaner.cancel()
- env.actorSystem.stop(heartbeatReceiver)
- cleaner.foreach(_.stop())
- dagSchedulerCopy.stop()
- taskScheduler = null
- // TODO: Cache.stop()?
- env.stop()
- SparkEnv.set(null)
- listenerBus.stop()
- eventLogger.foreach(_.stop())
- logInfo("Successfully stopped SparkContext")
- } else {
- logInfo("SparkContext already stopped")
+ SparkContext.SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
+ postApplicationEnd()
+ ui.foreach(_.stop())
+ // Do this only if not stopped already - best case effort.
+ // prevent NPE if stopped more than once.
+ val dagSchedulerCopy = dagScheduler
+ dagScheduler = null
+ if (dagSchedulerCopy != null) {
+ env.metricsSystem.report()
+ metadataCleaner.cancel()
+ env.actorSystem.stop(heartbeatReceiver)
+ cleaner.foreach(_.stop())
+ dagSchedulerCopy.stop()
+ taskScheduler = null
+ // TODO: Cache.stop()?
+ env.stop()
+ SparkEnv.set(null)
+ listenerBus.stop()
+ eventLogger.foreach(_.stop())
+ logInfo("Successfully stopped SparkContext")
+ SparkContext.clearActiveContext()
+ } else {
+ logInfo("SparkContext already stopped")
+ }
}
}
@@ -1257,6 +1281,7 @@ class SparkContext(config: SparkConf) extends Logging {
logInfo("Starting job: " + callSite.shortForm)
dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal,
resultHandler, localProperties.get)
+ progressBar.foreach(_.finishAll())
rdd.doCheckpoint()
}
@@ -1475,6 +1500,11 @@ class SparkContext(config: SparkConf) extends Logging {
private[spark] def cleanup(cleanupTime: Long) {
persistentRdds.clearOldValues(cleanupTime)
}
+
+ // In order to prevent multiple SparkContexts from being active at the same time, mark this
+ // context as having finished construction.
+ // NOTE: this must be placed at the end of the SparkContext constructor.
+ SparkContext.setActiveContext(this, allowMultipleContexts)
}
/**
@@ -1483,6 +1513,107 @@ class SparkContext(config: SparkConf) extends Logging {
*/
object SparkContext extends Logging {
+ /**
+ * Lock that guards access to global variables that track SparkContext construction.
+ */
+ private val SPARK_CONTEXT_CONSTRUCTOR_LOCK = new Object()
+
+ /**
+ * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `None`.
+ *
+ * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK
+ */
+ private var activeContext: Option[SparkContext] = None
+
+ /**
+ * Points to a partially-constructed SparkContext if some thread is in the SparkContext
+ * constructor, or `None` if no SparkContext is being constructed.
+ *
+ * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK
+ */
+ private var contextBeingConstructed: Option[SparkContext] = None
+
+ /**
+ * Called to ensure that no other SparkContext is running in this JVM.
+ *
+ * Throws an exception if a running context is detected and logs a warning if another thread is
+ * constructing a SparkContext. This warning is necessary because the current locking scheme
+ * prevents us from reliably distinguishing between cases where another context is being
+ * constructed and cases where another constructor threw an exception.
+ */
+ private def assertNoOtherContextIsRunning(
+ sc: SparkContext,
+ allowMultipleContexts: Boolean): Unit = {
+ SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
+ contextBeingConstructed.foreach { otherContext =>
+ if (otherContext ne sc) { // checks for reference equality
+ // Since otherContext might point to a partially-constructed context, guard against
+ // its creationSite field being null:
+ val otherContextCreationSite =
+ Option(otherContext.creationSite).map(_.longForm).getOrElse("unknown location")
+ val warnMsg = "Another SparkContext is being constructed (or threw an exception in its" +
+ " constructor). This may indicate an error, since only one SparkContext may be" +
+ " running in this JVM (see SPARK-2243)." +
+ s" The other SparkContext was created at:\n$otherContextCreationSite"
+ logWarning(warnMsg)
+ }
+
+ activeContext.foreach { ctx =>
+ val errMsg = "Only one SparkContext may be running in this JVM (see SPARK-2243)." +
+ " To ignore this error, set spark.driver.allowMultipleContexts = true. " +
+ s"The currently running SparkContext was created at:\n${ctx.creationSite.longForm}"
+ val exception = new SparkException(errMsg)
+ if (allowMultipleContexts) {
+ logWarning("Multiple running SparkContexts detected in the same JVM!", exception)
+ } else {
+ throw exception
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Called at the beginning of the SparkContext constructor to ensure that no SparkContext is
+ * running. Throws an exception if a running context is detected and logs a warning if another
+ * thread is constructing a SparkContext. This warning is necessary because the current locking
+ * scheme prevents us from reliably distinguishing between cases where another context is being
+ * constructed and cases where another constructor threw an exception.
+ */
+ private[spark] def markPartiallyConstructed(
+ sc: SparkContext,
+ allowMultipleContexts: Boolean): Unit = {
+ SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
+ assertNoOtherContextIsRunning(sc, allowMultipleContexts)
+ contextBeingConstructed = Some(sc)
+ }
+ }
+
+ /**
+ * Called at the end of the SparkContext constructor to ensure that no other SparkContext has
+ * raced with this constructor and started.
+ */
+ private[spark] def setActiveContext(
+ sc: SparkContext,
+ allowMultipleContexts: Boolean): Unit = {
+ SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
+ assertNoOtherContextIsRunning(sc, allowMultipleContexts)
+ contextBeingConstructed = None
+ activeContext = Some(sc)
+ }
+ }
+
+ /**
+ * Clears the active SparkContext metadata. This is called by `SparkContext#stop()`. It's
+ * also called in unit tests to prevent a flood of warnings from test suites that don't / can't
+ * properly clean up their SparkContexts.
+ */
+ private[spark] def clearActiveContext(): Unit = {
+ SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
+ activeContext = None
+ }
+ }
+
private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description"
private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id"
@@ -1682,6 +1813,9 @@ object SparkContext extends Logging {
def localCpuCount = Runtime.getRuntime.availableProcessors()
// local[*] estimates the number of cores on the machine; local[N] uses exactly N threads.
val threadCount = if (threads == "*") localCpuCount else threads.toInt
+ if (threadCount <= 0) {
+ throw new SparkException(s"Asked to run locally with $threadCount threads")
+ }
val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true)
val backend = new LocalBackend(scheduler, threadCount)
scheduler.initialize(backend)
diff --git a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala
index c18d763d7ff4d..edbdda8a0bcb6 100644
--- a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala
+++ b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala
@@ -96,6 +96,7 @@ class SparkStatusTracker private[spark] (sc: SparkContext) {
new SparkStageInfoImpl(
stageId,
info.attemptId,
+ info.submissionTime.getOrElse(0),
info.name,
info.numTasks,
data.numActiveTasks,
diff --git a/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala b/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala
index 90b47c847fbca..e5c7c8d0db578 100644
--- a/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala
+++ b/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala
@@ -26,6 +26,7 @@ private class SparkJobInfoImpl (
private class SparkStageInfoImpl(
val stageId: Int,
val currentAttemptId: Int,
+ val submissionTime: Long,
val name: String,
val numTasks: Int,
val numActiveTasks: Int,
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index d50ed32ca085c..6a6d9bf6857d3 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -42,6 +42,9 @@ import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD}
/**
* A Java-friendly version of [[org.apache.spark.SparkContext]] that returns
* [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones.
+ *
+ * Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before
+ * creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details.
*/
class JavaSparkContext(val sc: SparkContext)
extends JavaSparkContextVarargsWorkaround with Closeable {
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index 45beb8fc8c925..b80c771d58a8f 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -47,7 +47,7 @@ private[spark] class PythonRDD(
pythonIncludes: JList[String],
preservePartitoning: Boolean,
pythonExec: String,
- broadcastVars: JList[Broadcast[Array[Byte]]],
+ broadcastVars: JList[Broadcast[Array[Array[Byte]]]],
accumulator: Accumulator[JList[Array[Byte]]])
extends RDD[Array[Byte]](parent) {
@@ -230,8 +230,8 @@ private[spark] class PythonRDD(
if (!oldBids.contains(broadcast.id)) {
// send new broadcast
dataOut.writeLong(broadcast.id)
- dataOut.writeInt(broadcast.value.length)
- dataOut.write(broadcast.value)
+ dataOut.writeLong(broadcast.value.map(_.length.toLong).sum)
+ broadcast.value.foreach(dataOut.write)
oldBids.add(broadcast.id)
}
}
@@ -368,16 +368,24 @@ private[spark] object PythonRDD extends Logging {
}
}
- def readBroadcastFromFile(sc: JavaSparkContext, filename: String): Broadcast[Array[Byte]] = {
+ def readBroadcastFromFile(
+ sc: JavaSparkContext,
+ filename: String): Broadcast[Array[Array[Byte]]] = {
+ val size = new File(filename).length()
val file = new DataInputStream(new FileInputStream(filename))
+ val blockSize = 1 << 20
+ val n = ((size + blockSize - 1) / blockSize).toInt
+ val obj = new Array[Array[Byte]](n)
try {
- val length = file.readInt()
- val obj = new Array[Byte](length)
- file.readFully(obj)
- sc.broadcast(obj)
+ for (i <- 0 until n) {
+ val length = if (i < (n - 1)) blockSize else (size % blockSize).toInt
+ obj(i) = new Array[Byte](length)
+ file.readFully(obj(i))
+ }
} finally {
file.close()
}
+ sc.broadcast(obj)
}
def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) {
diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
index 39150deab863c..2e1e52906ceeb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
@@ -17,6 +17,8 @@
package org.apache.spark.deploy
+import java.net.{URI, URISyntaxException}
+
import scala.collection.mutable.ListBuffer
import org.apache.log4j.Level
@@ -73,7 +75,8 @@ private[spark] class ClientArguments(args: Array[String]) {
if (!ClientArguments.isValidJarUrl(_jarUrl)) {
println(s"Jar url '${_jarUrl}' is not in valid format.")
- println(s"Must be a jar file path in URL format (e.g. hdfs://XX.jar, file://XX.jar)")
+ println(s"Must be a jar file path in URL format " +
+ "(e.g. hdfs://host:port/XX.jar, file:///XX.jar)")
printUsageAndExit(-1)
}
@@ -114,5 +117,12 @@ private[spark] class ClientArguments(args: Array[String]) {
}
object ClientArguments {
- def isValidJarUrl(s: String): Boolean = s.matches("(.+):(.+)jar")
+ def isValidJarUrl(s: String): Boolean = {
+ try {
+ val uri = new URI(s)
+ uri.getScheme != null && uri.getPath != null && uri.getPath.endsWith(".jar")
+ } catch {
+ case _: URISyntaxException => false
+ }
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala
index aa3743ca7df63..d2687faad62b1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala
@@ -134,7 +134,7 @@ private[spark] object SparkSubmitDriverBootstrapper {
override def run() = {
if (process != null) {
process.destroy()
- sys.exit(process.waitFor())
+ process.waitFor()
}
}
})
diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala
index e2fc9c649925e..436dbed1730bc 100644
--- a/core/src/main/scala/org/apache/spark/package.scala
+++ b/core/src/main/scala/org/apache/spark/package.scala
@@ -44,5 +44,5 @@ package org.apache
package object spark {
// For package docs only
- val SPARK_VERSION = "1.2.0-SNAPSHOT"
+ val SPARK_VERSION = "1.3.0-SNAPSHOT"
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala
index e2c301603b4a5..8c43a559409f2 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala
@@ -39,21 +39,24 @@ class ZippedWithIndexRDDPartition(val prev: Partition, val startIndex: Long)
private[spark]
class ZippedWithIndexRDD[T: ClassTag](@transient prev: RDD[T]) extends RDD[(T, Long)](prev) {
- override def getPartitions: Array[Partition] = {
+ /** The start index of each partition. */
+ @transient private val startIndices: Array[Long] = {
val n = prev.partitions.size
- val startIndices: Array[Long] =
- if (n == 0) {
- Array[Long]()
- } else if (n == 1) {
- Array(0L)
- } else {
- prev.context.runJob(
- prev,
- Utils.getIteratorSize _,
- 0 until n - 1, // do not need to count the last partition
- false
- ).scanLeft(0L)(_ + _)
- }
+ if (n == 0) {
+ Array[Long]()
+ } else if (n == 1) {
+ Array(0L)
+ } else {
+ prev.context.runJob(
+ prev,
+ Utils.getIteratorSize _,
+ 0 until n - 1, // do not need to count the last partition
+ allowLocal = false
+ ).scanLeft(0L)(_ + _)
+ }
+ }
+
+ override def getPartitions: Array[Partition] = {
firstParent[T].partitions.map(x => new ZippedWithIndexRDDPartition(x, startIndices(x.index)))
}
diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala
index 6908a59a79e60..af873034215a9 100644
--- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala
@@ -148,6 +148,7 @@ private[spark] class TachyonBlockManager(
logError("Exception while deleting tachyon spark dir: " + tachyonDir, e)
}
}
+ client.close()
}
})
}
diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala
index 6dbad5ff0518e..233d1e2b7c616 100644
--- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala
@@ -116,6 +116,8 @@ private[spark] class TachyonStore(
case ioe: IOException =>
logWarning(s"Failed to fetch the block $blockId from Tachyon", ioe)
None
+ } finally {
+ is.close()
}
}
diff --git a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala
new file mode 100644
index 0000000000000..27ba9e18237b5
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala
@@ -0,0 +1,124 @@
+/*
+ * 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.ui
+
+import java.util.{Timer, TimerTask}
+
+import org.apache.spark._
+
+/**
+ * ConsoleProgressBar shows the progress of stages in the next line of the console. It poll the
+ * status of active stages from `sc.statusTracker` periodically, the progress bar will be showed
+ * up after the stage has ran at least 500ms. If multiple stages run in the same time, the status
+ * of them will be combined together, showed in one line.
+ */
+private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging {
+
+ // Carrige return
+ val CR = '\r'
+ // Update period of progress bar, in milliseconds
+ val UPDATE_PERIOD = 200L
+ // Delay to show up a progress bar, in milliseconds
+ val FIRST_DELAY = 500L
+
+ // The width of terminal
+ val TerminalWidth = if (!sys.env.getOrElse("COLUMNS", "").isEmpty) {
+ sys.env.get("COLUMNS").get.toInt
+ } else {
+ 80
+ }
+
+ var lastFinishTime = 0L
+ var lastUpdateTime = 0L
+ var lastProgressBar = ""
+
+ // Schedule a refresh thread to run periodically
+ private val timer = new Timer("refresh progress", true)
+ timer.schedule(new TimerTask{
+ override def run() {
+ refresh()
+ }
+ }, FIRST_DELAY, UPDATE_PERIOD)
+
+ /**
+ * Try to refresh the progress bar in every cycle
+ */
+ private def refresh(): Unit = synchronized {
+ val now = System.currentTimeMillis()
+ if (now - lastFinishTime < FIRST_DELAY) {
+ return
+ }
+ val stageIds = sc.statusTracker.getActiveStageIds()
+ val stages = stageIds.map(sc.statusTracker.getStageInfo).flatten.filter(_.numTasks() > 1)
+ .filter(now - _.submissionTime() > FIRST_DELAY).sortBy(_.stageId())
+ if (stages.size > 0) {
+ show(now, stages.take(3)) // display at most 3 stages in same time
+ }
+ }
+
+ /**
+ * Show progress bar in console. The progress bar is displayed in the next line
+ * after your last output, keeps overwriting itself to hold in one line. The logging will follow
+ * the progress bar, then progress bar will be showed in next line without overwrite logs.
+ */
+ private def show(now: Long, stages: Seq[SparkStageInfo]) {
+ val width = TerminalWidth / stages.size
+ val bar = stages.map { s =>
+ val total = s.numTasks()
+ val header = s"[Stage ${s.stageId()}:"
+ val tailer = s"(${s.numCompletedTasks()} + ${s.numActiveTasks()}) / $total]"
+ val w = width - header.size - tailer.size
+ val bar = if (w > 0) {
+ val percent = w * s.numCompletedTasks() / total
+ (0 until w).map { i =>
+ if (i < percent) "=" else if (i == percent) ">" else " "
+ }.mkString("")
+ } else {
+ ""
+ }
+ header + bar + tailer
+ }.mkString("")
+
+ // only refresh if it's changed of after 1 minute (or the ssh connection will be closed
+ // after idle some time)
+ if (bar != lastProgressBar || now - lastUpdateTime > 60 * 1000L) {
+ System.err.print(CR + bar)
+ lastUpdateTime = now
+ }
+ lastProgressBar = bar
+ }
+
+ /**
+ * Clear the progress bar if showed.
+ */
+ private def clear() {
+ if (!lastProgressBar.isEmpty) {
+ System.err.printf(CR + " " * TerminalWidth + CR)
+ lastProgressBar = ""
+ }
+ }
+
+ /**
+ * Mark all the stages as finished, clear the progress bar if showed, then the progress will not
+ * interweave with output of jobs.
+ */
+ def finishAll(): Unit = synchronized {
+ clear()
+ lastFinishTime = System.currentTimeMillis()
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
index be81ca6ab9be7..6ba80dbc3d95a 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
@@ -40,42 +40,110 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
import JobProgressListener._
+ // Define a handful of type aliases so that data structures' types can serve as documentation.
+ // These type aliases are public because they're used in the types of public fields:
+
type JobId = Int
type StageId = Int
type StageAttemptId = Int
+ type PoolName = String
+ type ExecutorId = String
- // How many stages to remember
- val retainedStages = conf.getInt("spark.ui.retainedStages", DEFAULT_RETAINED_STAGES)
- // How many jobs to remember
- val retailedJobs = conf.getInt("spark.ui.retainedJobs", DEFAULT_RETAINED_JOBS)
+ // Define all of our state:
+ // Jobs:
val activeJobs = new HashMap[JobId, JobUIData]
val completedJobs = ListBuffer[JobUIData]()
val failedJobs = ListBuffer[JobUIData]()
val jobIdToData = new HashMap[JobId, JobUIData]
+ // Stages:
val activeStages = new HashMap[StageId, StageInfo]
val completedStages = ListBuffer[StageInfo]()
val failedStages = ListBuffer[StageInfo]()
val stageIdToData = new HashMap[(StageId, StageAttemptId), StageUIData]
val stageIdToInfo = new HashMap[StageId, StageInfo]
val stageIdToActiveJobIds = new HashMap[StageId, HashSet[JobId]]
-
- // Number of completed and failed stages, may not actually equal to completedStages.size and
- // failedStages.size respectively due to completedStage and failedStages only maintain the latest
- // part of the stages, the earlier ones will be removed when there are too many stages for
- // memory sake.
+ val poolToActiveStages = HashMap[PoolName, HashMap[StageId, StageInfo]]()
+ // Total of completed and failed stages that have ever been run. These may be greater than
+ // `completedStages.size` and `failedStages.size` if we have run more stages or jobs than
+ // JobProgressListener's retention limits.
var numCompletedStages = 0
var numFailedStages = 0
- // Map from pool name to a hash map (map from stage id to StageInfo).
- val poolToActiveStages = HashMap[String, HashMap[Int, StageInfo]]()
-
- val executorIdToBlockManagerId = HashMap[String, BlockManagerId]()
+ // Misc:
+ val executorIdToBlockManagerId = HashMap[ExecutorId, BlockManagerId]()
+ def blockManagerIds = executorIdToBlockManagerId.values.toSeq
var schedulingMode: Option[SchedulingMode] = None
- def blockManagerIds = executorIdToBlockManagerId.values.toSeq
+ // To limit the total memory usage of JobProgressListener, we only track information for a fixed
+ // number of non-active jobs and stages (there is no limit for active jobs and stages):
+
+ val retainedStages = conf.getInt("spark.ui.retainedStages", DEFAULT_RETAINED_STAGES)
+ val retainedJobs = conf.getInt("spark.ui.retainedJobs", DEFAULT_RETAINED_JOBS)
+
+ // We can test for memory leaks by ensuring that collections that track non-active jobs and
+ // stages do not grow without bound and that collections for active jobs/stages eventually become
+ // empty once Spark is idle. Let's partition our collections into ones that should be empty
+ // once Spark is idle and ones that should have a hard- or soft-limited sizes.
+ // These methods are used by unit tests, but they're defined here so that people don't forget to
+ // update the tests when adding new collections. Some collections have multiple levels of
+ // nesting, etc, so this lets us customize our notion of "size" for each structure:
+
+ // These collections should all be empty once Spark is idle (no active stages / jobs):
+ private[spark] def getSizesOfActiveStateTrackingCollections: Map[String, Int] = {
+ Map(
+ "activeStages" -> activeStages.size,
+ "activeJobs" -> activeJobs.size,
+ "poolToActiveStages" -> poolToActiveStages.values.map(_.size).sum,
+ "stageIdToActiveJobIds" -> stageIdToActiveJobIds.values.map(_.size).sum
+ )
+ }
+
+ // These collections should stop growing once we have run at least `spark.ui.retainedStages`
+ // stages and `spark.ui.retainedJobs` jobs:
+ private[spark] def getSizesOfHardSizeLimitedCollections: Map[String, Int] = {
+ Map(
+ "completedJobs" -> completedJobs.size,
+ "failedJobs" -> failedJobs.size,
+ "completedStages" -> completedStages.size,
+ "failedStages" -> failedStages.size
+ )
+ }
+
+ // These collections may grow arbitrarily, but once Spark becomes idle they should shrink back to
+ // some bound based on the `spark.ui.retainedStages` and `spark.ui.retainedJobs` settings:
+ private[spark] def getSizesOfSoftSizeLimitedCollections: Map[String, Int] = {
+ Map(
+ "jobIdToData" -> jobIdToData.size,
+ "stageIdToData" -> stageIdToData.size,
+ "stageIdToStageInfo" -> stageIdToInfo.size
+ )
+ }
+
+ /** If stages is too large, remove and garbage collect old stages */
+ private def trimStagesIfNecessary(stages: ListBuffer[StageInfo]) = synchronized {
+ if (stages.size > retainedStages) {
+ val toRemove = math.max(retainedStages / 10, 1)
+ stages.take(toRemove).foreach { s =>
+ stageIdToData.remove((s.stageId, s.attemptId))
+ stageIdToInfo.remove(s.stageId)
+ }
+ stages.trimStart(toRemove)
+ }
+ }
+
+ /** If jobs is too large, remove and garbage collect old jobs */
+ private def trimJobsIfNecessary(jobs: ListBuffer[JobUIData]) = synchronized {
+ if (jobs.size > retainedJobs) {
+ val toRemove = math.max(retainedJobs / 10, 1)
+ jobs.take(toRemove).foreach { job =>
+ jobIdToData.remove(job.jobId)
+ }
+ jobs.trimStart(toRemove)
+ }
+ }
override def onJobStart(jobStart: SparkListenerJobStart) = synchronized {
val jobGroup = for (
@@ -113,9 +181,11 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
jobEnd.jobResult match {
case JobSucceeded =>
completedJobs += jobData
+ trimJobsIfNecessary(completedJobs)
jobData.status = JobExecutionStatus.SUCCEEDED
case JobFailed(exception) =>
failedJobs += jobData
+ trimJobsIfNecessary(failedJobs)
jobData.status = JobExecutionStatus.FAILED
}
for (stageId <- jobData.stageIds) {
@@ -142,11 +212,11 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
if (stage.failureReason.isEmpty) {
completedStages += stage
numCompletedStages += 1
- trimIfNecessary(completedStages)
+ trimStagesIfNecessary(completedStages)
} else {
failedStages += stage
numFailedStages += 1
- trimIfNecessary(failedStages)
+ trimStagesIfNecessary(failedStages)
}
for (
@@ -163,19 +233,6 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
}
}
- /** If stages is too large, remove and garbage collect old stages */
- private def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized {
- if (stages.size > retainedStages) {
- val toRemove = math.max(retainedStages / 10, 1)
- stages.take(toRemove).foreach { s =>
- stageIdToData.remove((s.stageId, s.attemptId))
- stageIdToInfo.remove(s.stageId)
- stageIdToActiveJobIds.remove(s.stageId)
- }
- stages.trimStart(toRemove)
- }
- }
-
/** For FIFO, all stages are contained by "default" pool but "default" pool here is meaningless */
override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized {
val stage = stageSubmitted.stageInfo
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index cd90b2358812e..40e05f86b661d 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -114,6 +114,10 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
+
+
+ (De)select All
+
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
index 26fa0cb6d7bde..8a0f5a602de12 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
@@ -76,10 +76,6 @@ class ExternalAppendOnlyMap[K, V, C](
private val sparkConf = SparkEnv.get.conf
private val diskBlockManager = blockManager.diskBlockManager
- // Number of pairs inserted since last spill; note that we count them even if a value is merged
- // with a previous key in case we're doing something like groupBy where the result grows
- protected[this] var elementsRead = 0L
-
/**
* Size of object batches when reading/writing from serializers.
*
@@ -132,7 +128,7 @@ class ExternalAppendOnlyMap[K, V, C](
currentMap = new SizeTrackingAppendOnlyMap[K, C]
}
currentMap.changeValue(curEntry._1, update)
- elementsRead += 1
+ addElementsRead()
}
}
@@ -209,8 +205,6 @@ class ExternalAppendOnlyMap[K, V, C](
}
spilledMaps.append(new DiskMapIterator(file, blockId, batchSizes))
-
- elementsRead = 0
}
def diskBytesSpilled: Long = _diskBytesSpilled
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
index c1ce13683b569..c617ff5c51d04 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
@@ -119,10 +119,6 @@ private[spark] class ExternalSorter[K, V, C](
private var map = new SizeTrackingAppendOnlyMap[(Int, K), C]
private var buffer = new SizeTrackingPairBuffer[(Int, K), C]
- // Number of pairs read from input since last spill; note that we count them even if a value is
- // merged with a previous key in case we're doing something like groupBy where the result grows
- protected[this] var elementsRead = 0L
-
// Total spilling statistics
private var _diskBytesSpilled = 0L
@@ -204,7 +200,7 @@ private[spark] class ExternalSorter[K, V, C](
if (hadValue) mergeValue(oldValue, kv._2) else createCombiner(kv._2)
}
while (records.hasNext) {
- elementsRead += 1
+ addElementsRead()
kv = records.next()
map.changeValue((getPartition(kv._1), kv._1), update)
maybeSpillCollection(usingMap = true)
@@ -212,7 +208,7 @@ private[spark] class ExternalSorter[K, V, C](
} else {
// Stick values into our buffer
while (records.hasNext) {
- elementsRead += 1
+ addElementsRead()
val kv = records.next()
buffer.insert((getPartition(kv._1), kv._1), kv._2.asInstanceOf[C])
maybeSpillCollection(usingMap = false)
diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala
index 0e4c6d633a4a9..cb73b377fca98 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala
@@ -36,7 +36,11 @@ private[spark] trait Spillable[C] {
protected def spill(collection: C): Unit
// Number of elements read from input since last spill
- protected var elementsRead: Long
+ protected def elementsRead: Long = _elementsRead
+
+ // Called by subclasses every time a record is read
+ // It's used for checking spilling frequency
+ protected def addElementsRead(): Unit = { _elementsRead += 1 }
// Memory manager that can be used to acquire/release memory
private[this] val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager
@@ -44,6 +48,9 @@ private[spark] trait Spillable[C] {
// What threshold of elementsRead we start estimating collection size at
private[this] val trackMemoryThreshold = 1000
+ // Number of elements read from input since last spill
+ private[this] var _elementsRead = 0L
+
// How much of the shared memory pool this collection has claimed
private[this] var myMemoryThreshold = 0L
@@ -76,6 +83,7 @@ private[spark] trait Spillable[C] {
spill(collection)
+ _elementsRead = 0
// Keep track of spills, and release memory
_memoryBytesSpilled += currentMemory
releaseMemoryForThisThread()
diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
index 4b27477790212..ce804f94f3267 100644
--- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
@@ -37,20 +37,24 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
.set("spark.dynamicAllocation.enabled", "true")
intercept[SparkException] { new SparkContext(conf) }
SparkEnv.get.stop() // cleanup the created environment
+ SparkContext.clearActiveContext()
// Only min
val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1")
intercept[SparkException] { new SparkContext(conf1) }
SparkEnv.get.stop()
+ SparkContext.clearActiveContext()
// Only max
val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2")
intercept[SparkException] { new SparkContext(conf2) }
SparkEnv.get.stop()
+ SparkContext.clearActiveContext()
// Both min and max, but min > max
intercept[SparkException] { createSparkContext(2, 1) }
SparkEnv.get.stop()
+ SparkContext.clearActiveContext()
// Both min and max, and min == max
val sc1 = createSparkContext(1, 1)
diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
index 31edad1c56c73..9e454ddcc52a6 100644
--- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
@@ -21,9 +21,62 @@ import org.scalatest.FunSuite
import org.apache.hadoop.io.BytesWritable
-class SparkContextSuite extends FunSuite {
- //Regression test for SPARK-3121
+class SparkContextSuite extends FunSuite with LocalSparkContext {
+
+ /** Allows system properties to be changed in tests */
+ private def withSystemProperty[T](property: String, value: String)(block: => T): T = {
+ val originalValue = System.getProperty(property)
+ try {
+ System.setProperty(property, value)
+ block
+ } finally {
+ if (originalValue == null) {
+ System.clearProperty(property)
+ } else {
+ System.setProperty(property, originalValue)
+ }
+ }
+ }
+
+ test("Only one SparkContext may be active at a time") {
+ // Regression test for SPARK-4180
+ withSystemProperty("spark.driver.allowMultipleContexts", "false") {
+ val conf = new SparkConf().setAppName("test").setMaster("local")
+ sc = new SparkContext(conf)
+ // A SparkContext is already running, so we shouldn't be able to create a second one
+ intercept[SparkException] { new SparkContext(conf) }
+ // After stopping the running context, we should be able to create a new one
+ resetSparkContext()
+ sc = new SparkContext(conf)
+ }
+ }
+
+ test("Can still construct a new SparkContext after failing to construct a previous one") {
+ withSystemProperty("spark.driver.allowMultipleContexts", "false") {
+ // This is an invalid configuration (no app name or master URL)
+ intercept[SparkException] {
+ new SparkContext(new SparkConf())
+ }
+ // Even though those earlier calls failed, we should still be able to create a new context
+ sc = new SparkContext(new SparkConf().setMaster("local").setAppName("test"))
+ }
+ }
+
+ test("Check for multiple SparkContexts can be disabled via undocumented debug option") {
+ withSystemProperty("spark.driver.allowMultipleContexts", "true") {
+ var secondSparkContext: SparkContext = null
+ try {
+ val conf = new SparkConf().setAppName("test").setMaster("local")
+ sc = new SparkContext(conf)
+ secondSparkContext = new SparkContext(conf)
+ } finally {
+ Option(secondSparkContext).foreach(_.stop())
+ }
+ }
+ }
+
test("BytesWritable implicit conversion is correct") {
+ // Regression test for SPARK-3121
val bytesWritable = new BytesWritable()
val inputArray = (1 to 10).map(_.toByte).toArray
bytesWritable.set(inputArray, 0, 10)
diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala
index 4161aede1d1d0..d2dae34be7bfb 100644
--- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala
@@ -23,12 +23,27 @@ import org.scalatest.Matchers
class ClientSuite extends FunSuite with Matchers {
test("correctly validates driver jar URL's") {
ClientArguments.isValidJarUrl("http://someHost:8080/foo.jar") should be (true)
- ClientArguments.isValidJarUrl("file://some/path/to/a/jarFile.jar") should be (true)
+
+ // file scheme with authority and path is valid.
+ ClientArguments.isValidJarUrl("file://somehost/path/to/a/jarFile.jar") should be (true)
+
+ // file scheme without path is not valid.
+ // In this case, jarFile.jar is recognized as authority.
+ ClientArguments.isValidJarUrl("file://jarFile.jar") should be (false)
+
+ // file scheme without authority but with triple slash is valid.
+ ClientArguments.isValidJarUrl("file:///some/path/to/a/jarFile.jar") should be (true)
ClientArguments.isValidJarUrl("hdfs://someHost:1234/foo.jar") should be (true)
ClientArguments.isValidJarUrl("hdfs://someHost:1234/foo") should be (false)
ClientArguments.isValidJarUrl("/missing/a/protocol/jarfile.jar") should be (false)
ClientArguments.isValidJarUrl("not-even-a-path.jar") should be (false)
+
+ // This URI doesn't have authority and path.
+ ClientArguments.isValidJarUrl("hdfs:someHost:1234/jarfile.jar") should be (false)
+
+ // Invalid syntax.
+ ClientArguments.isValidJarUrl("hdfs:") should be (false)
}
}
diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
index 6d2e696dc2fc4..e079ca3b1e896 100644
--- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
@@ -739,6 +739,11 @@ class RDDSuite extends FunSuite with SharedSparkContext {
}
}
+ test("zipWithIndex chained with other RDDs (SPARK-4433)") {
+ val count = sc.parallelize(0 until 10, 2).zipWithIndex().repartition(4).count()
+ assert(count === 10)
+ }
+
test("zipWithUniqueId") {
val n = 10
val data = sc.parallelize(0 until n, 3)
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 7c102cc7f4049..19fcb80d1db5c 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
@@ -28,32 +28,105 @@ import org.apache.spark.util.Utils
class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matchers {
- test("test LRU eviction of stages") {
- val conf = new SparkConf()
- conf.set("spark.ui.retainedStages", 5.toString)
- val listener = new JobProgressListener(conf)
- def createStageStartEvent(stageId: Int) = {
- val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, "")
- SparkListenerStageSubmitted(stageInfo)
+ private def createStageStartEvent(stageId: Int) = {
+ val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, "")
+ SparkListenerStageSubmitted(stageInfo)
+ }
+
+ private def createStageEndEvent(stageId: Int, failed: Boolean = false) = {
+ val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, "")
+ if (failed) {
+ stageInfo.failureReason = Some("Failed!")
}
+ SparkListenerStageCompleted(stageInfo)
+ }
- def createStageEndEvent(stageId: Int) = {
- val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, "")
- SparkListenerStageCompleted(stageInfo)
+ private def createJobStartEvent(jobId: Int, stageIds: Seq[Int]) = {
+ val stageInfos = stageIds.map { stageId =>
+ new StageInfo(stageId, 0, stageId.toString, 0, null, "")
}
+ SparkListenerJobStart(jobId, stageInfos, stageIds)
+ }
+
+ private def createJobEndEvent(jobId: Int, failed: Boolean = false) = {
+ val result = if (failed) JobFailed(new Exception("dummy failure")) else JobSucceeded
+ SparkListenerJobEnd(jobId, result)
+ }
+
+ private def runJob(listener: SparkListener, jobId: Int, shouldFail: Boolean = false) {
+ val stageIds = jobId * 100 to jobId * 100 + 50
+ listener.onJobStart(createJobStartEvent(jobId, stageIds))
+ for (stageId <- stageIds) {
+ listener.onStageSubmitted(createStageStartEvent(stageId))
+ listener.onStageCompleted(createStageEndEvent(stageId, failed = stageId % 2 == 0))
+ }
+ listener.onJobEnd(createJobEndEvent(jobId, shouldFail))
+ }
+
+ private def assertActiveJobsStateIsEmpty(listener: JobProgressListener) {
+ listener.getSizesOfActiveStateTrackingCollections.foreach { case (fieldName, size) =>
+ assert(size === 0, s"$fieldName was not empty")
+ }
+ }
+
+ test("test LRU eviction of stages") {
+ val conf = new SparkConf()
+ conf.set("spark.ui.retainedStages", 5.toString)
+ val listener = new JobProgressListener(conf)
for (i <- 1 to 50) {
listener.onStageSubmitted(createStageStartEvent(i))
listener.onStageCompleted(createStageEndEvent(i))
}
+ assertActiveJobsStateIsEmpty(listener)
listener.completedStages.size should be (5)
- listener.completedStages.count(_.stageId == 50) should be (1)
- listener.completedStages.count(_.stageId == 49) should be (1)
- listener.completedStages.count(_.stageId == 48) should be (1)
- listener.completedStages.count(_.stageId == 47) should be (1)
- listener.completedStages.count(_.stageId == 46) should be (1)
+ listener.completedStages.map(_.stageId).toSet should be (Set(50, 49, 48, 47, 46))
+ }
+
+ test("test LRU eviction of jobs") {
+ val conf = new SparkConf()
+ conf.set("spark.ui.retainedStages", 5.toString)
+ conf.set("spark.ui.retainedJobs", 5.toString)
+ val listener = new JobProgressListener(conf)
+
+ // Run a bunch of jobs to get the listener into a state where we've exceeded both the
+ // job and stage retention limits:
+ for (jobId <- 1 to 10) {
+ runJob(listener, jobId, shouldFail = false)
+ }
+ for (jobId <- 200 to 210) {
+ runJob(listener, jobId, shouldFail = true)
+ }
+ assertActiveJobsStateIsEmpty(listener)
+ // Snapshot the sizes of various soft- and hard-size-limited collections:
+ val softLimitSizes = listener.getSizesOfSoftSizeLimitedCollections
+ val hardLimitSizes = listener.getSizesOfHardSizeLimitedCollections
+ // Run some more jobs:
+ for (jobId <- 11 to 50) {
+ runJob(listener, jobId, shouldFail = false)
+ // We shouldn't exceed the hard / soft limit sizes after the jobs have finished:
+ listener.getSizesOfSoftSizeLimitedCollections should be (softLimitSizes)
+ listener.getSizesOfHardSizeLimitedCollections should be (hardLimitSizes)
+ }
+
+ listener.completedJobs.size should be (5)
+ listener.completedJobs.map(_.jobId).toSet should be (Set(50, 49, 48, 47, 46))
+
+ for (jobId <- 51 to 100) {
+ runJob(listener, jobId, shouldFail = true)
+ // We shouldn't exceed the hard / soft limit sizes after the jobs have finished:
+ listener.getSizesOfSoftSizeLimitedCollections should be (softLimitSizes)
+ listener.getSizesOfHardSizeLimitedCollections should be (hardLimitSizes)
+ }
+ assertActiveJobsStateIsEmpty(listener)
+
+ // Completed and failed jobs each their own size limits, so this should still be the same:
+ listener.completedJobs.size should be (5)
+ listener.completedJobs.map(_.jobId).toSet should be (Set(50, 49, 48, 47, 46))
+ listener.failedJobs.size should be (5)
+ listener.failedJobs.map(_.jobId).toSet should be (Set(100, 99, 98, 97, 96))
}
test("test executor id to summary") {
diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh
index a6e90a15ee84b..8a0b0348db8c0 100755
--- a/dev/create-release/create-release.sh
+++ b/dev/create-release/create-release.sh
@@ -28,13 +28,19 @@
# - Send output to stderr and have useful logging in stdout
# Note: The following variables must be set before use!
-GIT_USERNAME=${GIT_USERNAME:-pwendell}
-GIT_PASSWORD=${GIT_PASSWORD:-XXX}
+ASF_USERNAME=${ASF_USERNAME:-pwendell}
+ASF_PASSWORD=${ASF_PASSWORD:-XXX}
GPG_PASSPHRASE=${GPG_PASSPHRASE:-XXX}
GIT_BRANCH=${GIT_BRANCH:-branch-1.0}
-RELEASE_VERSION=${RELEASE_VERSION:-1.0.0}
+RELEASE_VERSION=${RELEASE_VERSION:-1.2.0}
+NEXT_VERSION=${NEXT_VERSION:-1.2.1}
RC_NAME=${RC_NAME:-rc2}
-USER_NAME=${USER_NAME:-pwendell}
+
+M2_REPO=~/.m2/repository
+SPARK_REPO=$M2_REPO/org/apache/spark
+NEXUS_ROOT=https://repository.apache.org/service/local/staging
+NEXUS_UPLOAD=$NEXUS_ROOT/deploy/maven2
+NEXUS_PROFILE=d63f592e7eac0 # Profile for Spark staging uploads
if [ -z "$JAVA_HOME" ]; then
echo "Error: JAVA_HOME is not set, cannot proceed."
@@ -47,31 +53,90 @@ set -e
GIT_TAG=v$RELEASE_VERSION-$RC_NAME
if [[ ! "$@" =~ --package-only ]]; then
- echo "Creating and publishing release"
+ echo "Creating release commit and publishing to Apache repository"
# Artifact publishing
- git clone https://git-wip-us.apache.org/repos/asf/spark.git -b $GIT_BRANCH
- cd spark
+ git clone https://$ASF_USERNAME:$ASF_PASSWORD@git-wip-us.apache.org/repos/asf/spark.git \
+ -b $GIT_BRANCH
+ pushd spark
export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g"
- mvn -Pyarn release:clean
-
- mvn -DskipTests \
- -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \
- -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \
- -Dmaven.javadoc.skip=true \
- -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \
- -Dtag=$GIT_TAG -DautoVersionSubmodules=true \
- -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \
- --batch-mode release:prepare
-
- mvn -DskipTests \
- -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \
- -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \
- -Dmaven.javadoc.skip=true \
+ # Create release commits and push them to github
+ # NOTE: This is done "eagerly" i.e. we don't check if we can succesfully build
+ # or before we coin the release commit. This helps avoid races where
+ # other people add commits to this branch while we are in the middle of building.
+ old=" ${RELEASE_VERSION}-SNAPSHOT<\/version>"
+ new=" ${RELEASE_VERSION}<\/version>"
+ find . -name pom.xml -o -name package.scala | grep -v dev | xargs -I {} sed -i \
+ -e "s/$old/$new/" {}
+ git commit -a -m "Preparing Spark release $GIT_TAG"
+ echo "Creating tag $GIT_TAG at the head of $GIT_BRANCH"
+ git tag $GIT_TAG
+
+ old=" ${RELEASE_VERSION}<\/version>"
+ new=" ${NEXT_VERSION}-SNAPSHOT<\/version>"
+ find . -name pom.xml -o -name package.scala | grep -v dev | xargs -I {} sed -i \
+ -e "s/$old/$new/" {}
+ git commit -a -m "Preparing development version ${NEXT_VERSION}-SNAPSHOT"
+ git push origin $GIT_TAG
+ git push origin HEAD:$GIT_BRANCH
+ git checkout -f $GIT_TAG
+
+ # Using Nexus API documented here:
+ # https://support.sonatype.com/entries/39720203-Uploading-to-a-Staging-Repository-via-REST-API
+ echo "Creating Nexus staging repository"
+ repo_request="Apache Spark $GIT_TAG"
+ out=$(curl -X POST -d "$repo_request" -u $ASF_USERNAME:$ASF_PASSWORD \
+ -H "Content-Type:application/xml" -v \
+ $NEXUS_ROOT/profiles/$NEXUS_PROFILE/start)
+ staged_repo_id=$(echo $out | sed -e "s/.*\(orgapachespark-[0-9]\{4\}\).*/\1/")
+ echo "Created Nexus staging repository: $staged_repo_id"
+
+ rm -rf $SPARK_REPO
+
+ mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \
-Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \
- release:perform
+ clean install
- cd ..
+ ./dev/change-version-to-2.11.sh
+
+ mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \
+ -Dscala-2.11 -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \
+ clean install
+
+ ./dev/change-version-to-2.10.sh
+
+ pushd $SPARK_REPO
+
+ # Remove any extra files generated during install
+ find . -type f |grep -v \.jar |grep -v \.pom | xargs rm
+
+ echo "Creating hash and signature files"
+ for file in $(find . -type f)
+ do
+ echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --output $file.asc --detach-sig --armour $file;
+ gpg --print-md MD5 $file > $file.md5;
+ gpg --print-md SHA1 $file > $file.sha1
+ done
+
+ echo "Uplading files to $NEXUS_UPLOAD"
+ for file in $(find . -type f)
+ do
+ # strip leading ./
+ file_short=$(echo $file | sed -e "s/\.\///")
+ dest_url="$NEXUS_UPLOAD/org/apache/spark/$file_short"
+ echo " Uploading $file_short"
+ curl -u $ASF_USERNAME:$ASF_PASSWORD --upload-file $file_short $dest_url
+ done
+
+ echo "Closing nexus staging repository"
+ repo_request="$staged_repo_idApache Spark $GIT_TAG"
+ out=$(curl -X POST -d "$repo_request" -u $ASF_USERNAME:$ASF_PASSWORD \
+ -H "Content-Type:application/xml" -v \
+ $NEXUS_ROOT/profiles/$NEXUS_PROFILE/finish)
+ echo "Closed Nexus staging repository: $staged_repo_id"
+
+ popd
+ popd
rm -rf spark
fi
@@ -102,6 +167,12 @@ make_binary_release() {
cp -r spark spark-$RELEASE_VERSION-bin-$NAME
cd spark-$RELEASE_VERSION-bin-$NAME
+
+ # TODO There should probably be a flag to make-distribution to allow 2.11 support
+ if [[ $FLAGS == *scala-2.11* ]]; then
+ ./dev/change-version-to-2.11.sh
+ fi
+
./make-distribution.sh --name $NAME --tgz $FLAGS 2>&1 | tee ../binary-release-$NAME.log
cd ..
cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz .
@@ -118,11 +189,12 @@ make_binary_release() {
spark-$RELEASE_VERSION-bin-$NAME.tgz.sha
}
+
make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" &
+make_binary_release "hadoop1-scala2.11" "-Phive -Dscala-2.11" &
make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" &
make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" &
make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" &
-make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" &
make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" &
make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" &
wait
@@ -130,10 +202,10 @@ wait
# Copy data
echo "Copying release tarballs"
rc_folder=spark-$RELEASE_VERSION-$RC_NAME
-ssh $USER_NAME@people.apache.org \
- mkdir /home/$USER_NAME/public_html/$rc_folder
+ssh $ASF_USERNAME@people.apache.org \
+ mkdir /home/$ASF_USERNAME/public_html/$rc_folder
scp spark-* \
- $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_folder/
+ $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_folder/
# Docs
cd spark
@@ -143,12 +215,12 @@ cd docs
JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build
echo "Copying release documentation"
rc_docs_folder=${rc_folder}-docs
-ssh $USER_NAME@people.apache.org \
- mkdir /home/$USER_NAME/public_html/$rc_docs_folder
-rsync -r _site/* $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_docs_folder
+ssh $ASF_USERNAME@people.apache.org \
+ mkdir /home/$ASF_USERNAME/public_html/$rc_docs_folder
+rsync -r _site/* $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_docs_folder
echo "Release $RELEASE_VERSION completed:"
echo "Git tag:\t $GIT_TAG"
echo "Release commit:\t $release_hash"
-echo "Binary location:\t http://people.apache.org/~$USER_NAME/$rc_folder"
-echo "Doc location:\t http://people.apache.org/~$USER_NAME/$rc_docs_folder"
+echo "Binary location:\t http://people.apache.org/~$ASF_USERNAME/$rc_folder"
+echo "Doc location:\t http://people.apache.org/~$ASF_USERNAME/$rc_docs_folder"
diff --git a/docs/_config.yml b/docs/_config.yml
index cdea02fcffbc5..a96a76dd9ab5e 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -13,8 +13,8 @@ include:
# These allow the documentation to be updated with newer releases
# of Spark, Scala, and Mesos.
-SPARK_VERSION: 1.2.0-SNAPSHOT
-SPARK_VERSION_SHORT: 1.2.0
+SPARK_VERSION: 1.3.0-SNAPSHOT
+SPARK_VERSION_SHORT: 1.3.0
SCALA_BINARY_VERSION: "2.10"
SCALA_VERSION: "2.10.4"
MESOS_VERSION: 0.18.1
diff --git a/docs/programming-guide.md b/docs/programming-guide.md
index 9de2f914b8b4c..49f319ba775e5 100644
--- a/docs/programming-guide.md
+++ b/docs/programming-guide.md
@@ -117,6 +117,8 @@ The first thing a Spark program must do is to create a [SparkContext](api/scala/
how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object
that contains information about your application.
+Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before creating a new one.
+
{% highlight scala %}
val conf = new SparkConf().setAppName(appName).setMaster(master)
new SparkContext(conf)
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 48e8267ac072c..5500da83b2b66 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -14,7 +14,7 @@ title: Spark SQL Programming Guide
Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using
Spark. At the core of this component is a new type of RDD,
[SchemaRDD](api/scala/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed of
-[Row](api/scala/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects, along with
+[Row](api/scala/index.html#org.apache.spark.sql.package@Row:org.apache.spark.sql.catalyst.expressions.Row.type) objects, along with
a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table
in a traditional relational database. A SchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io)
file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/).
diff --git a/examples/pom.xml b/examples/pom.xml
index 85e133779e465..8713230e1e8ed 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -21,7 +21,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../pom.xml
@@ -106,6 +106,11 @@
hbase-testing-util${hbase.version}
+
+
+ org.apache.hbase
+ hbase-annotations
+ org.jrubyjruby-complete
@@ -121,12 +126,24 @@
org.apache.hbasehbase-common${hbase.version}
+
+
+
+ org.apache.hbase
+ hbase-annotations
+
+ org.apache.hbasehbase-client${hbase.version}
+
+
+ org.apache.hbase
+ hbase-annotations
+ io.nettynetty
@@ -158,6 +175,11 @@
org.apache.hadoophadoop-auth
+
+
+ org.apache.hbase
+ hbase-annotations
+ org.apache.hadoophadoop-annotations
diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml
index ac291bd4fde20..72618b6515f83 100644
--- a/external/flume-sink/pom.xml
+++ b/external/flume-sink/pom.xml
@@ -21,7 +21,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../../pom.xml
diff --git a/external/flume/pom.xml b/external/flume/pom.xml
index 7d31e32283d88..a682f0e8471d8 100644
--- a/external/flume/pom.xml
+++ b/external/flume/pom.xml
@@ -21,7 +21,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../../pom.xml
@@ -39,19 +39,13 @@
org.apache.sparkspark-streaming_${scala.binary.version}${project.version}
+ providedorg.apache.sparkspark-streaming-flume-sink_${scala.binary.version}${project.version}
-
- org.apache.spark
- spark-streaming_${scala.binary.version}
- ${project.version}
- test-jar
- test
- org.apache.flumeflume-ng-sdk
diff --git a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
new file mode 100644
index 0000000000000..6e1f01900071b
--- /dev/null
+++ b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming;
+
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.junit.After;
+import org.junit.Before;
+
+public abstract class LocalJavaStreamingContext {
+
+ protected transient JavaStreamingContext ssc;
+
+ @Before
+ public void setUp() {
+ System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
+ ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
+ ssc.checkpoint("checkpoint");
+ }
+
+ @After
+ public void tearDown() {
+ ssc.stop();
+ ssc = null;
+ }
+}
diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala b/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala
new file mode 100644
index 0000000000000..1a900007b696b
--- /dev/null
+++ b/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming
+
+import java.io.{IOException, ObjectInputStream}
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.streaming.dstream.{DStream, ForEachDStream}
+import org.apache.spark.util.Utils
+
+import scala.collection.mutable.ArrayBuffer
+import scala.reflect.ClassTag
+
+/**
+ * This is a output stream just for the testsuites. All the output is collected into a
+ * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint.
+ *
+ * The buffer contains a sequence of RDD's, each containing a sequence of items
+ */
+class TestOutputStream[T: ClassTag](parent: DStream[T],
+ val output: ArrayBuffer[Seq[T]] = ArrayBuffer[Seq[T]]())
+ extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => {
+ val collected = rdd.collect()
+ output += collected
+ }) {
+
+ // This is to clear the output buffer every it is read from a checkpoint
+ @throws(classOf[IOException])
+ private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException {
+ ois.defaultReadObject()
+ output.clear()
+ }
+}
diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala
index 475026e8eb140..b57a1c71e35b9 100644
--- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala
+++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala
@@ -20,9 +20,6 @@ package org.apache.spark.streaming.flume
import java.net.InetSocketAddress
import java.util.concurrent.{Callable, ExecutorCompletionService, Executors}
-import java.util.Random
-
-import org.apache.spark.TestUtils
import scala.collection.JavaConversions._
import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer}
@@ -32,20 +29,35 @@ import org.apache.flume.channel.MemoryChannel
import org.apache.flume.conf.Configurables
import org.apache.flume.event.EventBuilder
+import org.scalatest.{BeforeAndAfter, FunSuite}
+
+import org.apache.spark.{SparkConf, Logging}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.dstream.ReceiverInputDStream
import org.apache.spark.streaming.util.ManualClock
-import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingContext}
+import org.apache.spark.streaming.{Seconds, TestOutputStream, StreamingContext}
import org.apache.spark.streaming.flume.sink._
import org.apache.spark.util.Utils
-class FlumePollingStreamSuite extends TestSuiteBase {
+class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging {
val batchCount = 5
val eventsPerBatch = 100
val totalEventsPerChannel = batchCount * eventsPerBatch
val channelCapacity = 5000
val maxAttempts = 5
+ val batchDuration = Seconds(1)
+
+ val conf = new SparkConf()
+ .setMaster("local[2]")
+ .setAppName(this.getClass.getSimpleName)
+
+ def beforeFunction() {
+ logInfo("Using manual clock")
+ conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+ }
+
+ before(beforeFunction())
test("flume polling test") {
testMultipleTimes(testFlumePolling)
@@ -229,4 +241,5 @@ class FlumePollingStreamSuite extends TestSuiteBase {
null
}
}
+
}
diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml
index 2067c473f0e3f..b3f44471cd326 100644
--- a/external/kafka/pom.xml
+++ b/external/kafka/pom.xml
@@ -21,7 +21,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../../pom.xml
@@ -39,13 +39,7 @@
org.apache.sparkspark-streaming_${scala.binary.version}${project.version}
-
-
- org.apache.spark
- spark-streaming_${scala.binary.version}
- ${project.version}
- test-jar
- test
+ providedorg.apache.kafka
diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml
index 362a76e515938..703806735b3ff 100644
--- a/external/mqtt/pom.xml
+++ b/external/mqtt/pom.xml
@@ -21,7 +21,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../../pom.xml
@@ -39,13 +39,7 @@
org.apache.sparkspark-streaming_${scala.binary.version}${project.version}
-
-
- org.apache.spark
- spark-streaming_${scala.binary.version}
- ${project.version}
- test-jar
- test
+ providedorg.eclipse.paho
diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
new file mode 100644
index 0000000000000..6e1f01900071b
--- /dev/null
+++ b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming;
+
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.junit.After;
+import org.junit.Before;
+
+public abstract class LocalJavaStreamingContext {
+
+ protected transient JavaStreamingContext ssc;
+
+ @Before
+ public void setUp() {
+ System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
+ ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
+ ssc.checkpoint("checkpoint");
+ }
+
+ @After
+ public void tearDown() {
+ ssc.stop();
+ ssc = null;
+ }
+}
diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
index 467fd263e2d64..84595acf45ccb 100644
--- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
+++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
@@ -17,11 +17,19 @@
package org.apache.spark.streaming.mqtt
-import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+import org.scalatest.FunSuite
+
+import org.apache.spark.streaming.{Seconds, StreamingContext}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.dstream.ReceiverInputDStream
-class MQTTStreamSuite extends TestSuiteBase {
+class MQTTStreamSuite extends FunSuite {
+
+ val batchDuration = Seconds(1)
+
+ private val master: String = "local[2]"
+
+ private val framework: String = this.getClass.getSimpleName
test("mqtt input stream") {
val ssc = new StreamingContext(master, framework, batchDuration)
diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml
index 1d7dd49d15c22..000ace1446e5e 100644
--- a/external/twitter/pom.xml
+++ b/external/twitter/pom.xml
@@ -21,7 +21,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../../pom.xml
@@ -39,13 +39,7 @@
org.apache.sparkspark-streaming_${scala.binary.version}${project.version}
-
-
- org.apache.spark
- spark-streaming_${scala.binary.version}
- ${project.version}
- test-jar
- test
+ providedorg.twitter4j
diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
new file mode 100644
index 0000000000000..6e1f01900071b
--- /dev/null
+++ b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming;
+
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.junit.After;
+import org.junit.Before;
+
+public abstract class LocalJavaStreamingContext {
+
+ protected transient JavaStreamingContext ssc;
+
+ @Before
+ public void setUp() {
+ System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
+ ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
+ ssc.checkpoint("checkpoint");
+ }
+
+ @After
+ public void tearDown() {
+ ssc.stop();
+ ssc = null;
+ }
+}
diff --git a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
index 93741e0375164..9ee57d7581d85 100644
--- a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
+++ b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
@@ -17,13 +17,23 @@
package org.apache.spark.streaming.twitter
-import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
-import org.apache.spark.storage.StorageLevel
+
+import org.scalatest.{BeforeAndAfter, FunSuite}
+import twitter4j.Status
import twitter4j.auth.{NullAuthorization, Authorization}
+
+import org.apache.spark.Logging
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.dstream.ReceiverInputDStream
-import twitter4j.Status
-class TwitterStreamSuite extends TestSuiteBase {
+class TwitterStreamSuite extends FunSuite with BeforeAndAfter with Logging {
+
+ val batchDuration = Seconds(1)
+
+ private val master: String = "local[2]"
+
+ private val framework: String = this.getClass.getSimpleName
test("twitter input stream") {
val ssc = new StreamingContext(master, framework, batchDuration)
diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml
index 7e48968feb3bc..29c452093502e 100644
--- a/external/zeromq/pom.xml
+++ b/external/zeromq/pom.xml
@@ -21,7 +21,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../../pom.xml
@@ -39,13 +39,7 @@
org.apache.sparkspark-streaming_${scala.binary.version}${project.version}
-
-
- org.apache.spark
- spark-streaming_${scala.binary.version}
- ${project.version}
- test-jar
- test
+ provided${akka.group}
diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
new file mode 100644
index 0000000000000..6e1f01900071b
--- /dev/null
+++ b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming;
+
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.junit.After;
+import org.junit.Before;
+
+public abstract class LocalJavaStreamingContext {
+
+ protected transient JavaStreamingContext ssc;
+
+ @Before
+ public void setUp() {
+ System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
+ ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
+ ssc.checkpoint("checkpoint");
+ }
+
+ @After
+ public void tearDown() {
+ ssc.stop();
+ ssc = null;
+ }
+}
diff --git a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
index cc10ff6ae03cd..a7566e733d891 100644
--- a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
+++ b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
@@ -20,12 +20,19 @@ package org.apache.spark.streaming.zeromq
import akka.actor.SupervisorStrategy
import akka.util.ByteString
import akka.zeromq.Subscribe
+import org.scalatest.FunSuite
import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+import org.apache.spark.streaming.{Seconds, StreamingContext}
import org.apache.spark.streaming.dstream.ReceiverInputDStream
-class ZeroMQStreamSuite extends TestSuiteBase {
+class ZeroMQStreamSuite extends FunSuite {
+
+ val batchDuration = Seconds(1)
+
+ private val master: String = "local[2]"
+
+ private val framework: String = this.getClass.getSimpleName
test("zeromq input stream") {
val ssc = new StreamingContext(master, framework, batchDuration)
diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml
index 7e478bed62da7..c8477a6566311 100644
--- a/extras/java8-tests/pom.xml
+++ b/extras/java8-tests/pom.xml
@@ -20,7 +20,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../../pom.xml
diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml
index 560244ad93369..c0d3a61119113 100644
--- a/extras/kinesis-asl/pom.xml
+++ b/extras/kinesis-asl/pom.xml
@@ -20,7 +20,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../../pom.xml
diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml
index 71a078d58a8d8..d1427f6a0c6e9 100644
--- a/extras/spark-ganglia-lgpl/pom.xml
+++ b/extras/spark-ganglia-lgpl/pom.xml
@@ -20,7 +20,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../../pom.xml
diff --git a/graphx/pom.xml b/graphx/pom.xml
index 3f49b1d63b6e1..9982b36f9b62f 100644
--- a/graphx/pom.xml
+++ b/graphx/pom.xml
@@ -21,7 +21,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../pom.xml
diff --git a/mllib/pom.xml b/mllib/pom.xml
index dd68b27a78bdc..0a6dda0ab8c80 100644
--- a/mllib/pom.xml
+++ b/mllib/pom.xml
@@ -21,7 +21,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../pom.xml
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 c8476a5370b6c..6f94b7f483ee0 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
@@ -229,6 +229,41 @@ class PythonMLLibAPI extends Serializable {
initialWeights)
}
+ /**
+ * Java stub for Python mllib LogisticRegressionWithLBFGS.train()
+ */
+ def trainLogisticRegressionModelWithLBFGS(
+ data: JavaRDD[LabeledPoint],
+ numIterations: Int,
+ initialWeights: Vector,
+ regParam: Double,
+ regType: String,
+ intercept: Boolean,
+ corrections: Int,
+ tolerance: Double): JList[Object] = {
+ val LogRegAlg = new LogisticRegressionWithLBFGS()
+ LogRegAlg.setIntercept(intercept)
+ LogRegAlg.optimizer
+ .setNumIterations(numIterations)
+ .setRegParam(regParam)
+ .setNumCorrections(corrections)
+ .setConvergenceTol(tolerance)
+ if (regType == "l2") {
+ LogRegAlg.optimizer.setUpdater(new SquaredL2Updater)
+ } else if (regType == "l1") {
+ LogRegAlg.optimizer.setUpdater(new L1Updater)
+ } else if (regType == null) {
+ LogRegAlg.optimizer.setUpdater(new SimpleUpdater)
+ } else {
+ throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter."
+ + " Can only be initialized using the following string values: ['l1', 'l2', None].")
+ }
+ trainRegressionModel(
+ LogRegAlg,
+ data,
+ initialWeights)
+ }
+
/**
* Java stub for NaiveBayes.train()
*/
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
index 18b95f1edc0b0..94d757bc317ab 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
@@ -64,7 +64,7 @@ class LogisticRegressionModel (
val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept
val score = 1.0 / (1.0 + math.exp(-margin))
threshold match {
- case Some(t) => if (score < t) 0.0 else 1.0
+ case Some(t) => if (score > t) 1.0 else 0.0
case None => score
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
index ab9515b2a6db8..dd514ff8a37f2 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
@@ -65,7 +65,7 @@ class SVMModel (
intercept: Double) = {
val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept
threshold match {
- case Some(t) => if (margin < t) 0.0 else 1.0
+ case Some(t) => if (margin > t) 1.0 else 0.0
case None => margin
}
}
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 038edc3521f14..90ac252226006 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
@@ -746,7 +746,7 @@ object ALS {
* @param iterations number of iterations of ALS (recommended: 10-20)
* @param lambda regularization factor (recommended: 0.01)
* @param blocks level of parallelism to split computation into
- * @param alpha confidence parameter (only applies when immplicitPrefs = true)
+ * @param alpha confidence parameter
* @param seed random seed
*/
def trainImplicit(
@@ -773,7 +773,7 @@ object ALS {
* @param iterations number of iterations of ALS (recommended: 10-20)
* @param lambda regularization factor (recommended: 0.01)
* @param blocks level of parallelism to split computation into
- * @param alpha confidence parameter (only applies when immplicitPrefs = true)
+ * @param alpha confidence parameter
*/
def trainImplicit(
ratings: RDD[Rating],
@@ -797,6 +797,7 @@ object ALS {
* @param rank number of features to use
* @param iterations number of iterations of ALS (recommended: 10-20)
* @param lambda regularization factor (recommended: 0.01)
+ * @param alpha confidence parameter
*/
def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double, alpha: Double)
: MatrixFactorizationModel = {
diff --git a/network/common/pom.xml b/network/common/pom.xml
index 2bd0a7d2945dd..baca859fa5011 100644
--- a/network/common/pom.xml
+++ b/network/common/pom.xml
@@ -22,7 +22,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../../pom.xml
diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml
index 12ff034cfe588..12468567c3aed 100644
--- a/network/shuffle/pom.xml
+++ b/network/shuffle/pom.xml
@@ -22,7 +22,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../../pom.xml
diff --git a/network/yarn/pom.xml b/network/yarn/pom.xml
index 7845011ec3200..acec8f18f2b5c 100644
--- a/network/yarn/pom.xml
+++ b/network/yarn/pom.xml
@@ -22,7 +22,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOT../../pom.xml
diff --git a/pom.xml b/pom.xml
index 639ea22a1fda3..6c1c1214a7d3e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -26,7 +26,7 @@
org.apache.sparkspark-parent
- 1.2.0-SNAPSHOT
+ 1.3.0-SNAPSHOTpomSpark Project Parent POMhttp://spark.apache.org/
@@ -254,6 +254,18 @@
false
+
+
+ spark-staging-1038
+ Spark 1.2.0 Staging (1038)
+ https://repository.apache.org/content/repositories/orgapachespark-1038/
+
+ true
+
+
+ false
+
+
@@ -269,7 +281,7 @@
-
@@ -278,7 +290,7 @@
unused1.0.0
-