aboutsummaryrefslogtreecommitdiff
path: root/streaming
diff options
context:
space:
mode:
authorMatei Zaharia <matei@eecs.berkeley.edu>2013-02-17 23:23:08 -0800
committerMatei Zaharia <matei@eecs.berkeley.edu>2013-02-17 23:23:08 -0800
commit7151e1e4c8f4f764c54047ef82b988f887a0b9c7 (patch)
tree2ecfad2243345019d94828be4e69a1b52d33e6aa /streaming
parent06e5e6627f3856b5c6e3e60cbb167044de9ef6d4 (diff)
downloadspark-7151e1e4c8f4f764c54047ef82b988f887a0b9c7.tar.gz
spark-7151e1e4c8f4f764c54047ef82b988f887a0b9c7.tar.bz2
spark-7151e1e4c8f4f764c54047ef82b988f887a0b9c7.zip
Rename "jobs" to "applications" in the standalone cluster
Diffstat (limited to 'streaming')
-rw-r--r--streaming/src/main/scala/spark/streaming/Checkpoint.scala2
-rw-r--r--streaming/src/main/scala/spark/streaming/StreamingContext.scala10
-rw-r--r--streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala6
3 files changed, 9 insertions, 9 deletions
diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
index 2f3adb39c2..80244520a3 100644
--- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
@@ -12,7 +12,7 @@ private[streaming]
class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
extends Logging with Serializable {
val master = ssc.sc.master
- val framework = ssc.sc.jobName
+ val framework = ssc.sc.appName
val sparkHome = ssc.sc.sparkHome
val jars = ssc.sc.jars
val graph = ssc.graph
diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala
index 37ba524b48..0cce2b13cf 100644
--- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala
@@ -39,11 +39,11 @@ class StreamingContext private (
/**
* Creates a StreamingContext by providing the details necessary for creating a new SparkContext.
* @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
- * @param frameworkName A name for your job, to display on the cluster web UI
+ * @param appName A name for your job, to display on the cluster web UI
* @param batchDuration The time interval at which streaming data will be divided into batches
*/
- def this(master: String, frameworkName: String, batchDuration: Duration) =
- this(StreamingContext.createNewSparkContext(master, frameworkName), null, batchDuration)
+ def this(master: String, appName: String, batchDuration: Duration) =
+ this(StreamingContext.createNewSparkContext(master, appName), null, batchDuration)
/**
* Re-creates a StreamingContext from a checkpoint file.
@@ -384,14 +384,14 @@ object StreamingContext {
new PairDStreamFunctions[K, V](stream)
}
- protected[streaming] def createNewSparkContext(master: String, frameworkName: String): SparkContext = {
+ protected[streaming] def createNewSparkContext(master: String, appName: String): SparkContext = {
// Set the default cleaner delay to an hour if not already set.
// This should be sufficient for even 1 second interval.
if (MetadataCleaner.getDelaySeconds < 0) {
MetadataCleaner.setDelaySeconds(3600)
}
- new SparkContext(master, frameworkName)
+ new SparkContext(master, appName)
}
protected[streaming] def rddToFileName[T](prefix: String, suffix: String, time: Time): String = {
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
index e7f446a49b..e5b5e9ac23 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
@@ -27,11 +27,11 @@ class JavaStreamingContext(val ssc: StreamingContext) {
/**
* Creates a StreamingContext.
* @param master Name of the Spark Master
- * @param frameworkName Name to be used when registering with the scheduler
+ * @param appName Name to be used when registering with the scheduler
* @param batchDuration The time interval at which streaming data will be divided into batches
*/
- def this(master: String, frameworkName: String, batchDuration: Duration) =
- this(new StreamingContext(master, frameworkName, batchDuration))
+ def this(master: String, appName: String, batchDuration: Duration) =
+ this(new StreamingContext(master, appName, batchDuration))
/**
* Creates a StreamingContext.