Skip to content

Commit 0d61ee8

Browse files
committed
Merge branch 'streaming-web-ui' of github.com:tdas/spark into ui-refactor
2 parents 9a48fa1 + 61358e3 commit 0d61ee8

File tree

17 files changed

+706
-95
lines changed

17 files changed

+706
-95
lines changed

core/pom.xml

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -82,10 +82,6 @@
8282
<groupId>com.google.guava</groupId>
8383
<artifactId>guava</artifactId>
8484
</dependency>
85-
<dependency>
86-
<groupId>com.google.code.findbugs</groupId>
87-
<artifactId>jsr305</artifactId>
88-
</dependency>
8985
<dependency>
9086
<groupId>org.slf4j</groupId>
9187
<artifactId>slf4j-api</artifactId>

pom.xml

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -214,11 +214,6 @@
214214
<artifactId>guava</artifactId>
215215
<version>14.0.1</version>
216216
</dependency>
217-
<dependency>
218-
<groupId>com.google.code.findbugs</groupId>
219-
<artifactId>jsr305</artifactId>
220-
<version>1.3.9</version>
221-
</dependency>
222217
<dependency>
223218
<groupId>org.slf4j</groupId>
224219
<artifactId>slf4j-api</artifactId>

project/SparkBuild.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -296,7 +296,6 @@ object SparkBuild extends Build {
296296
name := "spark-core",
297297
libraryDependencies ++= Seq(
298298
"com.google.guava" % "guava" % "14.0.1",
299-
"com.google.code.findbugs" % "jsr305" % "1.3.9",
300299
"log4j" % "log4j" % "1.2.17",
301300
"org.slf4j" % "slf4j-api" % slf4jVersion,
302301
"org.slf4j" % "slf4j-log4j12" % slf4jVersion,

streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,7 @@ import org.apache.spark.streaming.dstream._
4040
import org.apache.spark.streaming.receivers._
4141
import org.apache.spark.streaming.scheduler._
4242
import org.apache.hadoop.conf.Configuration
43+
import org.apache.spark.streaming.ui.StreamingUI
4344

4445
/**
4546
* Main entry point for Spark Streaming functionality. It provides methods used to create
@@ -158,6 +159,9 @@ class StreamingContext private[streaming] (
158159

159160
private[streaming] val waiter = new ContextWaiter
160161

162+
private[streaming] val ui = new StreamingUI(this)
163+
ui.bind()
164+
161165
/**
162166
* Return the associated Spark context
163167
*/

streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala

Lines changed: 0 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -351,15 +351,6 @@ abstract class DStream[T: ClassTag] (
351351
dependencies.foreach(_.clearMetadata(time))
352352
}
353353

354-
/* Adds metadata to the Stream while it is running.
355-
* This method should be overwritten by sublcasses of InputDStream.
356-
*/
357-
private[streaming] def addMetadata(metadata: Any) {
358-
if (metadata != null) {
359-
logInfo("Dropping Metadata: " + metadata.toString)
360-
}
361-
}
362-
363354
/**
364355
* Refresh the list of checkpointed RDDs that will be saved along with checkpoint of
365356
* this stream. This is an internal method that should not be called directly. This is

streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala

Lines changed: 56 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -17,23 +17,24 @@
1717

1818
package org.apache.spark.streaming.dstream
1919

20-
import java.util.concurrent.ArrayBlockingQueue
2120
import java.nio.ByteBuffer
21+
import java.util.concurrent.ArrayBlockingQueue
2222

23-
import scala.collection.mutable.ArrayBuffer
23+
import scala.collection.mutable.{ArrayBuffer, HashMap}
2424
import scala.concurrent.Await
2525
import scala.concurrent.duration._
2626
import scala.reflect.ClassTag
2727

28-
import akka.actor.{Props, Actor}
28+
import akka.actor.{Actor, Props}
2929
import akka.pattern.ask
3030

31-
import org.apache.spark.streaming.util.{RecurringTimer, SystemClock}
32-
import org.apache.spark.streaming._
3331
import org.apache.spark.{Logging, SparkEnv}
34-
import org.apache.spark.rdd.{RDD, BlockRDD}
32+
import org.apache.spark.rdd.{BlockRDD, RDD}
3533
import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId}
36-
import org.apache.spark.streaming.scheduler.{DeregisterReceiver, AddBlocks, RegisterReceiver}
34+
import org.apache.spark.streaming._
35+
import org.apache.spark.streaming.scheduler.{ReceivedBlockInfo, AddBlocks, DeregisterReceiver, RegisterReceiver}
36+
import org.apache.spark.streaming.util.{RecurringTimer, SystemClock}
37+
import org.apache.spark.util.Utils
3738

3839
/**
3940
* Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]]
@@ -48,8 +49,10 @@ import org.apache.spark.streaming.scheduler.{DeregisterReceiver, AddBlocks, Regi
4849
abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingContext)
4950
extends InputDStream[T](ssc_) {
5051

51-
// This is an unique identifier that is used to match the network receiver with the
52-
// corresponding network input stream.
52+
/** Keeps all received blocks information */
53+
private val receivedBlockInfo = new HashMap[Time, Array[ReceivedBlockInfo]]
54+
55+
/** This is an unique identifier for the network input stream. */
5356
val id = ssc.getNewNetworkStreamId()
5457

5558
/**
@@ -64,23 +67,45 @@ abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingConte
6467

6568
def stop() {}
6669

70+
/** Ask NetworkInputTracker for received data blocks and generates RDDs with them. */
6771
override def compute(validTime: Time): Option[RDD[T]] = {
6872
// If this is called for any time before the start time of the context,
6973
// then this returns an empty RDD. This may happen when recovering from a
7074
// master failure
7175
if (validTime >= graph.startTime) {
72-
val blockIds = ssc.scheduler.networkInputTracker.getBlockIds(id, validTime)
76+
val blockInfo = ssc.scheduler.networkInputTracker.getReceivedBlockInfo(id)
77+
receivedBlockInfo(validTime) = blockInfo
78+
val blockIds = blockInfo.map(_.blockId.asInstanceOf[BlockId])
7379
Some(new BlockRDD[T](ssc.sc, blockIds))
7480
} else {
7581
Some(new BlockRDD[T](ssc.sc, Array[BlockId]()))
7682
}
7783
}
84+
85+
/** Get information on received blocks. */
86+
private[streaming] def getReceivedBlockInfo(time: Time) = {
87+
receivedBlockInfo(time)
88+
}
89+
90+
/**
91+
* Clear metadata that are older than `rememberDuration` of this DStream.
92+
* This is an internal method that should not be called directly. This
93+
* implementation overrides the default implementation to clear received
94+
* block information.
95+
*/
96+
private[streaming] override def clearMetadata(time: Time) {
97+
super.clearMetadata(time)
98+
val oldReceivedBlocks = receivedBlockInfo.filter(_._1 <= (time - rememberDuration))
99+
receivedBlockInfo --= oldReceivedBlocks.keys
100+
logDebug("Cleared " + oldReceivedBlocks.size + " RDDs that were older than " +
101+
(time - rememberDuration) + ": " + oldReceivedBlocks.keys.mkString(", "))
102+
}
78103
}
79104

80105

81106
private[streaming] sealed trait NetworkReceiverMessage
82107
private[streaming] case class StopReceiver(msg: String) extends NetworkReceiverMessage
83-
private[streaming] case class ReportBlock(blockId: BlockId, metadata: Any)
108+
private[streaming] case class ReportBlock(blockId: StreamBlockId, numRecords: Long, metadata: Any)
84109
extends NetworkReceiverMessage
85110
private[streaming] case class ReportError(msg: String) extends NetworkReceiverMessage
86111

@@ -155,21 +180,30 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging
155180
actor ! ReportError(e.toString)
156181
}
157182

158-
159183
/**
160184
* Pushes a block (as an ArrayBuffer filled with data) into the block manager.
161185
*/
162-
def pushBlock(blockId: BlockId, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) {
186+
def pushBlock(
187+
blockId: StreamBlockId,
188+
arrayBuffer: ArrayBuffer[T],
189+
metadata: Any,
190+
level: StorageLevel
191+
) {
163192
env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level)
164-
actor ! ReportBlock(blockId, metadata)
193+
actor ! ReportBlock(blockId, arrayBuffer.size, metadata)
165194
}
166195

167196
/**
168197
* Pushes a block (as bytes) into the block manager.
169198
*/
170-
def pushBlock(blockId: BlockId, bytes: ByteBuffer, metadata: Any, level: StorageLevel) {
199+
def pushBlock(
200+
blockId: StreamBlockId,
201+
bytes: ByteBuffer,
202+
metadata: Any,
203+
level: StorageLevel
204+
) {
171205
env.blockManager.putBytes(blockId, bytes, level)
172-
actor ! ReportBlock(blockId, metadata)
206+
actor ! ReportBlock(blockId, -1 , metadata)
173207
}
174208

175209
/** A helper actor that communicates with the NetworkInputTracker */
@@ -182,13 +216,15 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging
182216
val timeout = 5.seconds
183217

184218
override def preStart() {
185-
val future = tracker.ask(RegisterReceiver(streamId, self))(timeout)
219+
val msg = RegisterReceiver(
220+
streamId, NetworkReceiver.this.getClass.getSimpleName, Utils.localHostName(), self)
221+
val future = tracker.ask(msg)(timeout)
186222
Await.result(future, timeout)
187223
}
188224

189225
override def receive() = {
190-
case ReportBlock(blockId, metadata) =>
191-
tracker ! AddBlocks(streamId, Array(blockId), metadata)
226+
case ReportBlock(blockId, numRecords, metadata) =>
227+
tracker ! AddBlocks(ReceivedBlockInfo(streamId, blockId, numRecords, metadata))
192228
case ReportError(msg) =>
193229
tracker ! DeregisterReceiver(streamId, msg)
194230
case StopReceiver(msg) =>
@@ -210,7 +246,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging
210246
class BlockGenerator(storageLevel: StorageLevel)
211247
extends Serializable with Logging {
212248

213-
case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null)
249+
case class Block(id: StreamBlockId, buffer: ArrayBuffer[T], metadata: Any = null)
214250

215251
val clock = new SystemClock()
216252
val blockInterval = env.conf.getLong("spark.streaming.blockInterval", 200)

streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ import org.apache.spark.streaming.Time
2929
*/
3030
case class BatchInfo(
3131
batchTime: Time,
32+
receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]],
3233
submissionTime: Long,
3334
processingStartTime: Option[Long],
3435
processingEndTime: Option[Long]

streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -147,7 +147,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
147147
logInfo("Batches to reschedule (" + timesToReschedule.size + " batches): " +
148148
timesToReschedule.mkString(", "))
149149
timesToReschedule.foreach(time =>
150-
jobScheduler.runJobs(time, graph.generateJobs(time))
150+
jobScheduler.submitJobSet(JobSet(time, graph.generateJobs(time)))
151151
)
152152

153153
// Restart the timer
@@ -159,7 +159,13 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
159159
private def generateJobs(time: Time) {
160160
SparkEnv.set(ssc.env)
161161
Try(graph.generateJobs(time)) match {
162-
case Success(jobs) => jobScheduler.runJobs(time, jobs)
162+
case Success(jobs) =>
163+
val receivedBlockInfo = graph.getNetworkInputStreams.map { stream =>
164+
val streamId = stream.id
165+
val receivedBlockInfo = stream.getReceivedBlockInfo(time)
166+
(streamId, receivedBlockInfo)
167+
}.toMap
168+
jobScheduler.submitJobSet(JobSet(time, jobs, receivedBlockInfo))
163169
case Failure(e) => jobScheduler.reportError("Error generating jobs for time " + time, e)
164170
}
165171
eventActor ! DoCheckpoint(time)

streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -82,14 +82,13 @@ class JobScheduler(val ssc: StreamingContext) extends Logging {
8282
}
8383
}
8484

85-
def runJobs(time: Time, jobs: Seq[Job]) {
86-
if (jobs.isEmpty) {
87-
logInfo("No jobs added for time " + time)
85+
def submitJobSet(jobSet: JobSet) {
86+
if (jobSet.jobs.isEmpty) {
87+
logInfo("No jobs added for time " + jobSet.time)
8888
} else {
89-
val jobSet = new JobSet(time, jobs)
90-
jobSets.put(time, jobSet)
89+
jobSets.put(jobSet.time, jobSet)
9190
jobSet.jobs.foreach(job => executor.execute(new JobHandler(job)))
92-
logInfo("Added jobs for time " + time)
91+
logInfo("Added jobs for time " + jobSet.time)
9392
}
9493
}
9594

streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,11 @@ import org.apache.spark.streaming.Time
2424
* belong to the same batch.
2525
*/
2626
private[streaming]
27-
case class JobSet(time: Time, jobs: Seq[Job]) {
27+
case class JobSet(
28+
time: Time,
29+
jobs: Seq[Job],
30+
receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]] = Map.empty
31+
) {
2832

2933
private val incompleteJobs = new HashSet[Job]()
3034
private val submissionTime = System.currentTimeMillis() // when this jobset was submitted
@@ -60,6 +64,7 @@ case class JobSet(time: Time, jobs: Seq[Job]) {
6064
def toBatchInfo: BatchInfo = {
6165
new BatchInfo(
6266
time,
67+
receivedBlockInfo,
6368
submissionTime,
6469
if (processingStartTime >= 0 ) Some(processingStartTime) else None,
6570
if (processingEndTime >= 0 ) Some(processingEndTime) else None

0 commit comments

Comments
 (0)