aboutsummaryrefslogtreecommitdiff
path: root/streaming/src/main/scala/spark/streaming/StreamingContext.scala
blob: 998fea849f81b1592fa31df385523e121939c866 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
package spark.streaming

import spark.RDD
import spark.Logging
import spark.SparkEnv
import spark.SparkContext
import spark.storage.StorageLevel

import scala.collection.mutable.Queue

import java.io.InputStream
import java.util.concurrent.atomic.AtomicInteger

import org.apache.hadoop.io.LongWritable
import org.apache.hadoop.io.Text
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat
import org.apache.flume.source.avro.AvroFlumeEvent
import org.apache.hadoop.fs.Path
import java.util.UUID
import spark.util.MetadataCleaner

/**
 * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
 * information (such as, cluster URL and job name) to internally create a SparkContext, it provides
 * methods used to create DStream from various input sources.
 */
class StreamingContext private (
    sc_ : SparkContext,
    cp_ : Checkpoint,
    batchDur_ : Time
  ) extends Logging {

  /**
   * Creates a StreamingContext using an existing SparkContext.
   * @param sparkContext Existing SparkContext
   * @param batchDuration The time interval at which streaming data will be divided into batches
   */
  def this(sparkContext: SparkContext, batchDuration: Time) = this(sparkContext, null, batchDuration)

  /**
   * 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 batchDuration The time interval at which streaming data will be divided into batches
   */
  def this(master: String, frameworkName: String, batchDuration: Time) =
    this(StreamingContext.createNewSparkContext(master, frameworkName), null, batchDuration)

  /**
   * Re-creates a StreamingContext from a checkpoint file.
   * @param path Path either to the directory that was specified as the checkpoint directory, or
   *             to the checkpoint file 'graph' or 'graph.bk'.
   */
  def this(path: String) = this(null, CheckpointReader.read(path), null)

  initLogging()

  if (sc_ == null && cp_ == null) {
    throw new Exception("Streaming Context cannot be initilalized with " +
      "both SparkContext and checkpoint as null")
  }

  protected[streaming] val isCheckpointPresent = (cp_ != null)

  val sc: SparkContext = {
    if (isCheckpointPresent) {
      new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars)
    } else {
      sc_
    }
  }

  protected[streaming] val env = SparkEnv.get

  protected[streaming] val graph: DStreamGraph = {
    if (isCheckpointPresent) {
      cp_.graph.setContext(this)
      cp_.graph.restoreCheckpointData()
      cp_.graph
    } else {
      assert(batchDur_ != null, "Batch duration for streaming context cannot be null")
      val newGraph = new DStreamGraph()
      newGraph.setBatchDuration(batchDur_)
      newGraph
    }
  }

  protected[streaming] val nextNetworkInputStreamId = new AtomicInteger(0)
  protected[streaming] var networkInputTracker: NetworkInputTracker = null

  protected[streaming] var checkpointDir: String = {
    if (isCheckpointPresent) {
      sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(cp_.checkpointDir), true)
      cp_.checkpointDir
    } else {
      null
    }
  }

  protected[streaming] var checkpointInterval: Time = if (isCheckpointPresent) cp_.checkpointInterval else null
  protected[streaming] var receiverJobThread: Thread = null
  protected[streaming] var scheduler: Scheduler = null

  def remember(duration: Time) {
    graph.remember(duration)
  }

  def checkpoint(dir: String, interval: Time = null) {
    if (dir != null) {
      sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(dir))
      checkpointDir = dir
      checkpointInterval = interval
    } else {
      checkpointDir = null
      checkpointInterval = null
    }
  }

  protected[streaming] def getInitialCheckpoint(): Checkpoint = {
    if (isCheckpointPresent) cp_ else null
  }

  protected[streaming] def getNewNetworkStreamId() = nextNetworkInputStreamId.getAndIncrement()

 /**
   * Create an input stream that pulls messages form a Kafka Broker.
   * 
   * @param host Zookeper hostname.
   * @param port Zookeper port.
   * @param groupId The group id for this consumer.
   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
   * in its own thread.
   * @param initialOffsets Optional initial offsets for each of the partitions to consume.
   * By default the value is pulled from zookeper.
   * @param storageLevel RDD storage level. Defaults to memory-only.
   */
  def kafkaStream[T: ClassManifest](
      hostname: String,
      port: Int,
      groupId: String,
      topics: Map[String, Int],
      initialOffsets: Map[KafkaPartitionKey, Long] = Map[KafkaPartitionKey, Long](),
      storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2
    ): DStream[T] = {
    val inputStream = new KafkaInputDStream[T](this, hostname, port, groupId, topics, initialOffsets, storageLevel)
    graph.addInputStream(inputStream)
    inputStream
  }

  def networkTextStream(
      hostname: String,
      port: Int,
      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
    ): DStream[String] = {
    networkStream[String](hostname, port, SocketReceiver.bytesToLines, storageLevel)
  }

  def networkStream[T: ClassManifest](
      hostname: String,
      port: Int,
      converter: (InputStream) => Iterator[T],
      storageLevel: StorageLevel
    ): DStream[T] = {
    val inputStream = new SocketInputDStream[T](this, hostname, port, converter, storageLevel)
    graph.addInputStream(inputStream)
    inputStream
  }

  def flumeStream (
    hostname: String,
    port: Int,
    storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2): DStream[SparkFlumeEvent] = {
    val inputStream = new FlumeInputDStream(this, hostname, port, storageLevel)
    graph.addInputStream(inputStream)
    inputStream
  }


  def rawNetworkStream[T: ClassManifest](
      hostname: String,
      port: Int,
      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
    ): DStream[T] = {
    val inputStream = new RawInputDStream[T](this, hostname, port, storageLevel)
    graph.addInputStream(inputStream)
    inputStream
  }

  /**
   * Creates a input stream that monitors a Hadoop-compatible filesystem
   * for new files and executes the necessary processing on them.
   */
  def fileStream[
    K: ClassManifest,
    V: ClassManifest,
    F <: NewInputFormat[K, V]: ClassManifest
  ](directory: String): DStream[(K, V)] = {
    val inputStream = new FileInputDStream[K, V, F](this, directory)
    graph.addInputStream(inputStream)
    inputStream
  }

  def textFileStream(directory: String): DStream[String] = {
    fileStream[LongWritable, Text, TextInputFormat](directory).map(_._2.toString)
  }

  /**
   * Creates a input stream from an queue of RDDs. In each batch,
   * it will process either one or all of the RDDs returned by the queue
   */
  def queueStream[T: ClassManifest](
      queue: Queue[RDD[T]],
      oneAtATime: Boolean = true,
      defaultRDD: RDD[T] = null
    ): DStream[T] = {
    val inputStream = new QueueInputDStream(this, queue, oneAtATime, defaultRDD)
    graph.addInputStream(inputStream)
    inputStream
  }

  def queueStream[T: ClassManifest](array: Array[RDD[T]]): DStream[T] = {
    val queue = new Queue[RDD[T]]
    val inputStream = queueStream(queue, true, null)
    queue ++= array
    inputStream
  }

  def union[T: ClassManifest](streams: Seq[DStream[T]]): DStream[T] = {
    new UnionDStream[T](streams.toArray)
  }

  /**
   * Registers an input stream that will be started (InputDStream.start() called) to get the
   * input data.
   */
  def registerInputStream(inputStream: InputDStream[_]) {
    graph.addInputStream(inputStream)
  }

  /**
   * Registers an output stream that will be computed every interval
   */
  def registerOutputStream(outputStream: DStream[_]) {
    graph.addOutputStream(outputStream)
  }

  protected def validate() {
    assert(graph != null, "Graph is null")
    graph.validate()

    assert(
      checkpointDir == null || checkpointInterval != null,
      "Checkpoint directory has been set, but the graph checkpointing interval has " +
        "not been set. Please use StreamingContext.checkpoint() to set the interval."
    )
  }

  /**
   * This function starts the execution of the streams.
   */
  def start() {
    if (checkpointDir != null && checkpointInterval == null && graph != null) {
      checkpointInterval = graph.batchDuration
    }

    validate()

    val networkInputStreams = graph.getInputStreams().filter(s => s match {
        case n: NetworkInputDStream[_] => true
        case _ => false
      }).map(_.asInstanceOf[NetworkInputDStream[_]]).toArray

    if (networkInputStreams.length > 0) {
      // Start the network input tracker (must start before receivers)
      networkInputTracker = new NetworkInputTracker(this, networkInputStreams)
      networkInputTracker.start()
    }

    Thread.sleep(1000)

    // Start the scheduler
    scheduler = new Scheduler(this)
    scheduler.start()
  }

  /**
   * This function stops the execution of the streams.
   */
  def stop() {
    try {
      if (scheduler != null) scheduler.stop()
      if (networkInputTracker != null) networkInputTracker.stop()
      if (receiverJobThread != null) receiverJobThread.interrupt()
      sc.stop()
      logInfo("StreamingContext stopped successfully")
    } catch {
      case e: Exception => logWarning("Error while stopping", e)
    }
  }
}


object StreamingContext {

  protected[streaming] def createNewSparkContext(master: String, frameworkName: 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(60)
    }
    new SparkContext(master, frameworkName)
  }

  implicit def toPairDStreamFunctions[K: ClassManifest, V: ClassManifest](stream: DStream[(K,V)]) = {
    new PairDStreamFunctions[K, V](stream)
  }

  protected[streaming] def rddToFileName[T](prefix: String, suffix: String, time: Time): String = {
    if (prefix == null) {
      time.millis.toString
    } else if (suffix == null || suffix.length ==0) {
      prefix + "-" + time.milliseconds
    } else {
      prefix + "-" + time.milliseconds + "." + suffix
    }
  }

  protected[streaming] def getSparkCheckpointDir(sscCheckpointDir: String): String = {
    new Path(sscCheckpointDir, UUID.randomUUID.toString).toString
  }
}