├── spark-streaming-StreamingSource.adoc ├── spark-streaming-kafka-ConsumerStrategies.adoc ├── images ├── spark-streaming-jobscheduler.png ├── spark-streaming-jobset-states.png ├── spark-streaming-kafka-webui-jobs.png ├── spark-streaming-receivertracker.png ├── spark-streaming-streamingcontext.png ├── spark-streaming-updateStateByKey.png ├── spark-streaming-JobGenerator-start.png ├── spark-streaming-JobScheduler-start.png ├── spark-streaming-webui-streaming-tab.png ├── spark-streaming-StateDStream-compute.png ├── spark-streaming-StreamingContext-start.png ├── spark-streaming-StreamingContext-stop.png ├── spark-streaming-batch-processing-time.png ├── spark-streaming-kafka-webui-streaming.png ├── spark-streaming-webui-completed-batches.png ├── spark-streaming-jobset-generatejobs-event.png ├── spark-streaming-kafka-webui-details-batch.png ├── spark-streaming-webui-streaming-statistics.png ├── spark-streaming-kafka-0-10-webui-details-batch.png ├── spark-streaming-webui-scheduling-delay-increase.png ├── spark-streaming-webui-streaming-page-scheduling-delay.png ├── spark-streaming-JobGenerator-ClearCheckpointData-event.png ├── spark-streaming-webui-kafka-directinputstream-two-jobs.png └── spark-streaming-webui-streaming-tab-kafka-directstream-two-batches.png ├── graffles ├── spark-streaming-jobscheduler.graffle ├── spark-streaming-jobset-states.graffle ├── spark-streaming-receivertracker.graffle ├── spark-streaming-streamingcontext.graffle ├── spark-streaming-updateStateByKey.graffle ├── spark-streaming-JobGenerator-start.graffle ├── spark-streaming-JobScheduler-start.graffle ├── spark-streaming-StateDStream-compute.graffle ├── spark-streaming-StreamingContext-stop.graffle ├── spark-streaming-batch-processing-time.graffle ├── spark-streaming-StreamingContext-start.graffle ├── spark-streaming-jobset-generatejobs-event.graffle └── spark-streaming-JobGenerator-ClearCheckpointData-event.graffle ├── spark-streaming-ExecutorAllocationManager.adoc ├── book.json ├── spark-streaming-foreachdstreams.adoc ├── spark-streaming-dynamic-allocation.adoc ├── spark-streaming-constantinputdstreams.adoc ├── spark-streaming-operators-saveas.adoc ├── spark-streaming-transformeddstreams.adoc ├── spark-streaming-kafka-LocationStrategy.adoc ├── book-intro.adoc ├── spark-streaming-PairDStreamFunctions.adoc ├── spark-streaming-windoweddstreams.adoc ├── spark-streaming-receivers.adoc ├── spark-streaming-receivedblockhandlers.adoc ├── spark-streaming-mapwithstatedstreams.adoc ├── SUMMARY.adoc ├── spark-streaming-kafka-HasOffsetRanges.adoc ├── spark-streaming-inputdstreams.adoc ├── spark-streaming-streaminglisteners.adoc ├── spark-streaming-receiverinputdstreams.adoc ├── spark-streaming-settings.adoc ├── spark-streaming.adoc ├── spark-streaming-windowedoperators.adoc ├── spark-streaming-kafka-KafkaUtils.adoc ├── spark-streaming-recurringtimer.adoc ├── spark-streaming-InputInfoTracker.adoc ├── spark-streaming-webui.adoc ├── spark-streaming-kafka.adoc ├── spark-streaming-kafka-KafkaRDD.adoc ├── spark-streaming-statedstreams.adoc ├── spark-streaming-backpressure.adoc ├── spark-streaming-kafka-ConsumerStrategy.adoc ├── spark-streaming-operators-stateful.adoc ├── spark-streaming-receiversupervisors.adoc ├── spark-streaming-operators.adoc ├── spark-streaming-receivertracker.adoc ├── spark-streaming-dstreamgraph.adoc ├── LICENSE ├── spark-streaming-jobscheduler.adoc ├── spark-streaming-kafka-DirectKafkaInputDStream.adoc └── spark-streaming-streamingcontext.adoc /spark-streaming-StreamingSource.adoc: -------------------------------------------------------------------------------- 1 | == [[StreamingSource]] StreamingSource 2 | 3 | CAUTION: FIXME 4 | -------------------------------------------------------------------------------- /spark-streaming-kafka-ConsumerStrategies.adoc: -------------------------------------------------------------------------------- 1 | == [[ConsumerStrategies]] ConsumerStrategies Factory Object 2 | -------------------------------------------------------------------------------- /images/spark-streaming-jobscheduler.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-jobscheduler.png -------------------------------------------------------------------------------- /images/spark-streaming-jobset-states.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-jobset-states.png -------------------------------------------------------------------------------- /images/spark-streaming-kafka-webui-jobs.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-kafka-webui-jobs.png -------------------------------------------------------------------------------- /images/spark-streaming-receivertracker.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-receivertracker.png -------------------------------------------------------------------------------- /images/spark-streaming-streamingcontext.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-streamingcontext.png -------------------------------------------------------------------------------- /images/spark-streaming-updateStateByKey.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-updateStateByKey.png -------------------------------------------------------------------------------- /graffles/spark-streaming-jobscheduler.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/graffles/spark-streaming-jobscheduler.graffle -------------------------------------------------------------------------------- /graffles/spark-streaming-jobset-states.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/graffles/spark-streaming-jobset-states.graffle -------------------------------------------------------------------------------- /images/spark-streaming-JobGenerator-start.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-JobGenerator-start.png -------------------------------------------------------------------------------- /images/spark-streaming-JobScheduler-start.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-JobScheduler-start.png -------------------------------------------------------------------------------- /images/spark-streaming-webui-streaming-tab.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-webui-streaming-tab.png -------------------------------------------------------------------------------- /graffles/spark-streaming-receivertracker.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/graffles/spark-streaming-receivertracker.graffle -------------------------------------------------------------------------------- /graffles/spark-streaming-streamingcontext.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/graffles/spark-streaming-streamingcontext.graffle -------------------------------------------------------------------------------- /graffles/spark-streaming-updateStateByKey.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/graffles/spark-streaming-updateStateByKey.graffle -------------------------------------------------------------------------------- /images/spark-streaming-StateDStream-compute.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-StateDStream-compute.png -------------------------------------------------------------------------------- /images/spark-streaming-StreamingContext-start.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-StreamingContext-start.png -------------------------------------------------------------------------------- /images/spark-streaming-StreamingContext-stop.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-StreamingContext-stop.png -------------------------------------------------------------------------------- /images/spark-streaming-batch-processing-time.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-batch-processing-time.png -------------------------------------------------------------------------------- /images/spark-streaming-kafka-webui-streaming.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-kafka-webui-streaming.png -------------------------------------------------------------------------------- /graffles/spark-streaming-JobGenerator-start.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/graffles/spark-streaming-JobGenerator-start.graffle -------------------------------------------------------------------------------- /graffles/spark-streaming-JobScheduler-start.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/graffles/spark-streaming-JobScheduler-start.graffle -------------------------------------------------------------------------------- /images/spark-streaming-webui-completed-batches.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-webui-completed-batches.png -------------------------------------------------------------------------------- /graffles/spark-streaming-StateDStream-compute.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/graffles/spark-streaming-StateDStream-compute.graffle -------------------------------------------------------------------------------- /graffles/spark-streaming-StreamingContext-stop.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/graffles/spark-streaming-StreamingContext-stop.graffle -------------------------------------------------------------------------------- /graffles/spark-streaming-batch-processing-time.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/graffles/spark-streaming-batch-processing-time.graffle -------------------------------------------------------------------------------- /images/spark-streaming-jobset-generatejobs-event.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-jobset-generatejobs-event.png -------------------------------------------------------------------------------- /images/spark-streaming-kafka-webui-details-batch.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-kafka-webui-details-batch.png -------------------------------------------------------------------------------- /images/spark-streaming-webui-streaming-statistics.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-webui-streaming-statistics.png -------------------------------------------------------------------------------- /graffles/spark-streaming-StreamingContext-start.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/graffles/spark-streaming-StreamingContext-start.graffle -------------------------------------------------------------------------------- /graffles/spark-streaming-jobset-generatejobs-event.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/graffles/spark-streaming-jobset-generatejobs-event.graffle -------------------------------------------------------------------------------- /images/spark-streaming-kafka-0-10-webui-details-batch.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-kafka-0-10-webui-details-batch.png -------------------------------------------------------------------------------- /images/spark-streaming-webui-scheduling-delay-increase.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-webui-scheduling-delay-increase.png -------------------------------------------------------------------------------- /spark-streaming-ExecutorAllocationManager.adoc: -------------------------------------------------------------------------------- 1 | == ExecutorAllocationManager 2 | 3 | CAUTION: FIXME 4 | 5 | === [[requestExecutors]] requestExecutors 6 | 7 | === [[killExecutor]] killExecutor 8 | -------------------------------------------------------------------------------- /images/spark-streaming-webui-streaming-page-scheduling-delay.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-webui-streaming-page-scheduling-delay.png -------------------------------------------------------------------------------- /images/spark-streaming-JobGenerator-ClearCheckpointData-event.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-JobGenerator-ClearCheckpointData-event.png -------------------------------------------------------------------------------- /images/spark-streaming-webui-kafka-directinputstream-two-jobs.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-webui-kafka-directinputstream-two-jobs.png -------------------------------------------------------------------------------- /graffles/spark-streaming-JobGenerator-ClearCheckpointData-event.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/graffles/spark-streaming-JobGenerator-ClearCheckpointData-event.graffle -------------------------------------------------------------------------------- /images/spark-streaming-webui-streaming-tab-kafka-directstream-two-batches.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/spark-streaming-notebook/HEAD/images/spark-streaming-webui-streaming-tab-kafka-directstream-two-batches.png -------------------------------------------------------------------------------- /book.json: -------------------------------------------------------------------------------- 1 | { 2 | "structure": { 3 | "readme": "book-intro.adoc" 4 | }, 5 | "variables": { 6 | "spark.version": "2.1.0", 7 | "sourcedir": "src/main/scala" 8 | }, 9 | "plugins": ["ga"], 10 | "pluginsConfig": { 11 | "ga": { 12 | "token": "UA-86782445-4" 13 | } 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /spark-streaming-foreachdstreams.adoc: -------------------------------------------------------------------------------- 1 | == ForEachDStreams 2 | 3 | `ForEachDStream` is an internal link:spark-streaming-dstreams.adoc[DStream] with dependency on the `parent` stream with the exact same `slideDuration`. 4 | 5 | The `compute` method returns no RDD. 6 | 7 | When `generateJob` is called, it returns a streaming job for a batch when `parent` stream does. And if so, it uses the "foreach" function (given as `foreachFunc`) to work on the RDDs generated. 8 | 9 | [NOTE] 10 | ==== 11 | Although it may seem that `ForEachDStreams` are by design output streams they are not. You have to use link:spark-streaming-dstreamgraph.adoc[DStreamGraph.addOutputStream] to register a stream as output. 12 | 13 | You use link:spark-streaming-operators.adoc[stream operators] that do the registration as part of their operation, like `print`. 14 | ==== 15 | -------------------------------------------------------------------------------- /spark-streaming-dynamic-allocation.adoc: -------------------------------------------------------------------------------- 1 | == Elastic Scaling (Dynamic Allocation) 2 | 3 | *Dynamic Allocation* in Spark Streaming makes for *adaptive streaming applications* by scaling them up and down to adapt to load variations. It actively controls resources (as executors) and prevents resources from being wasted when the processing time is short (comparing to a batch interval) - *scale down* - or adds new executors to decrease the processing time - *scale up*. 4 | 5 | NOTE: It is a work in progress in Spark Streaming and should be available in Spark 2.0. 6 | 7 | The motivation is to control the number of executors required to process input records when their number increases to the point when the link:spark-streaming-webui.adoc#processing-time[processing time] could become longer than the link:spark-streaming.adoc#batch-interval[batch interval]. 8 | 9 | === Configuration 10 | 11 | * `spark.streaming.dynamicAllocation.enabled` controls whether to enabled dynamic allocation (`true`) or not (`false`). 12 | -------------------------------------------------------------------------------- /spark-streaming-constantinputdstreams.adoc: -------------------------------------------------------------------------------- 1 | == ConstantInputDStreams 2 | 3 | `ConstantInputDStream` is an link:spark-streaming-inputdstreams.adoc[input stream] that always returns the same mandatory input RDD at every batch `time`. 4 | 5 | [source, scala] 6 | ---- 7 | ConstantInputDStream[T](_ssc: StreamingContext, rdd: RDD[T]) 8 | ---- 9 | 10 | `ConstantInputDStream` dstream belongs to `org.apache.spark.streaming.dstream` package. 11 | 12 | The `compute` method returns the input `rdd`. 13 | 14 | NOTE: `rdd` input parameter is mandatory. 15 | 16 | The mandatory `start` and `stop` methods do nothing. 17 | 18 | === [[example]] Example 19 | 20 | [source, scala] 21 | ---- 22 | val sc = new SparkContext("local[*]", "Constant Input DStream Demo", new SparkConf()) 23 | import org.apache.spark.streaming.{ StreamingContext, Seconds } 24 | val ssc = new StreamingContext(sc, batchDuration = Seconds(5)) 25 | 26 | // Create the RDD 27 | val rdd = sc.parallelize(0 to 9) 28 | 29 | // Create constant input dstream with the RDD 30 | import org.apache.spark.streaming.dstream.ConstantInputDStream 31 | val cis = new ConstantInputDStream(ssc, rdd) 32 | 33 | // Sample stream computation 34 | cis.print 35 | ---- 36 | -------------------------------------------------------------------------------- /spark-streaming-operators-saveas.adoc: -------------------------------------------------------------------------------- 1 | == SaveAs Operators 2 | 3 | There are two *saveAs operators* in link:spark-streaming-operators.adoc[DStream]: 4 | 5 | * `saveAsObjectFiles` 6 | * `saveAsTextFiles` 7 | 8 | They are link:spark-streaming-operators.adoc[output operators] that return nothing as they save each RDD in a batch to a storage. 9 | 10 | Their full signature is as follows: 11 | 12 | [source, scala] 13 | ---- 14 | saveAsObjectFiles(prefix: String, suffix: String = ""): Unit 15 | saveAsTextFiles(prefix: String, suffix: String = ""): Unit 16 | ---- 17 | 18 | NOTE: SaveAs operators use link:spark-streaming-operators.adoc#foreachRDD[foreachRDD] output operator. 19 | 20 | `saveAsObjectFiles` uses link:spark-io.adoc#saving-rdds-to-files[RDD.saveAsObjectFile] while `saveAsTextFiles` uses link:spark-io.adoc#saving-rdds-to-files[RDD.saveAsTextFile]. 21 | 22 | The file name is based on mandatory `prefix` and batch `time` with optional `suffix`. It is in the format of `[prefix]-[time in milliseconds].[suffix]`. 23 | 24 | === [[example]] Example 25 | 26 | [source, scala] 27 | ---- 28 | val clicks: InputDStream[(String, String)] = messages 29 | clicks.saveAsTextFiles("clicks", "txt") 30 | ---- 31 | -------------------------------------------------------------------------------- /spark-streaming-transformeddstreams.adoc: -------------------------------------------------------------------------------- 1 | == TransformedDStream 2 | 3 | `TransformedDStream` is the specialized link:spark-streaming-dstreams.adoc[DStream] that is the result of link:spark-streaming-operators.adoc#transform[transform] operator. 4 | 5 | It is constructed with a collection of `parents` dstreams and `transformFunc` transform function. 6 | 7 | NOTE: When created, it asserts that the input collection of dstreams use the same StreamingContext and slide interval. 8 | 9 | NOTE: It is acceptable to have more than one dependent dstream. 10 | 11 | The link:spark-streaming-dstreams.adoc#contract[dependencies] is the input collection of dstreams. 12 | 13 | The link:spark-streaming-dstreams.adoc#contract[slide interval] is exactly the same as that in the first dstream in `parents`. 14 | 15 | When requested to link:spark-streaming-dstreams.adoc#contract[compute a RDD], it goes over every dstream in `parents` and asks to getOrCompute a RDD. 16 | 17 | NOTE: It may throw a `SparkException` when a dstream does not compute a RDD for a batch. 18 | 19 | CAUTION: FIXME Prepare an example to face the exception. 20 | 21 | It then calls `transformFunc` with the collection of RDDs. 22 | 23 | If the transform function returns `null` a SparkException is thrown: 24 | 25 | [options="wrap"] 26 | ---- 27 | org.apache.spark.SparkException: Transform function must not return null. Return SparkContext.emptyRDD() instead to represent no element as the result of transformation. 28 | at org.apache.spark.streaming.dstream.TransformedDStream.compute(TransformedDStream.scala:48) 29 | ---- 30 | 31 | The result of `transformFunc` is returned. 32 | -------------------------------------------------------------------------------- /spark-streaming-kafka-LocationStrategy.adoc: -------------------------------------------------------------------------------- 1 | == [[LocationStrategy]] LocationStrategy -- Preferred Hosts per Topic Partitions 2 | 3 | `LocationStrategy` allows a link:spark-streaming-kafka-DirectKafkaInputDStream.adoc[DirectKafkaInputDStream] to request Spark executors to execute Kafka consumers as close topic leaders of topic partitions as possible. 4 | 5 | `LocationStrategy` is used when link:spark-streaming-kafka-DirectKafkaInputDStream.adoc#compute[`DirectKafkaInputDStream` computes a `KafkaRDD` for a given batch interval] and is a means of distributing processing Kafka records across Spark executors. 6 | 7 | .Location Strategies in Spark Streaming 8 | [cols="1,2",options="header",width="100%"] 9 | |=== 10 | | Location Strategy | Description 11 | | PreferBrokers 12 | | Use when executors are on the same nodes as your Kafka brokers. 13 | 14 | | PreferConsistent 15 | | Use in most cases as it consistently distributes partitions across all executors. 16 | 17 | | PreferFixed 18 | | Use to place particular ``TopicPartition``s on particular hosts if your load is uneven. 19 | 20 | Accepts a collection of topic partition and host pairs. Any topic partition not specified uses a consistent location. 21 | |=== 22 | 23 | NOTE: A topic partition is described using Kafka's https://kafka.apache.org/0100/javadoc/org/apache/kafka/common/TopicPartition.html[TopicPartition]. 24 | 25 | You can create a `LocationStrategy` using <>. 26 | 27 | [source, scala] 28 | ---- 29 | import org.apache.spark.streaming.kafka010.LocationStrategies 30 | val preferredHosts = LocationStrategies.PreferConsistent 31 | ---- 32 | 33 | === [[LocationStrategies]] LocationStrategies Factory Object 34 | 35 | `LocationStrategies` holds the factory methods to access `LocationStrategy` objects. 36 | 37 | [source, scala] 38 | ---- 39 | PreferBrokers: LocationStrategy 40 | PreferConsistent: LocationStrategy 41 | PreferFixed(hostMap: collection.Map[TopicPartition, String]): LocationStrategy 42 | ---- 43 | -------------------------------------------------------------------------------- /book-intro.adoc: -------------------------------------------------------------------------------- 1 | == Spark Streaming 2 | 3 | Welcome to Spark Streaming! 4 | 5 | I'm https://pl.linkedin.com/in/jaceklaskowski[Jacek Laskowski], an *independent consultant* who is passionate about *Apache Spark*, Apache Kafka, Scala, sbt (with some flavour of Apache Mesos, Hadoop YARN, and DC/OS). I lead http://www.meetup.com/WarsawScala/[Warsaw Scala Enthusiasts] and http://www.meetup.com/Warsaw-Spark[Warsaw Spark] meetups in Warsaw, Poland. 6 | 7 | Contact me at jacek@japila.pl or https://twitter.com/jaceklaskowski[@jaceklaskowski] to discuss Apache Spark opportunities, e.g. courses, workshops, mentoring or application development services. 8 | 9 | If you like the Apache Spark notes you should seriously consider participating in my own, very hands-on https://github.com/jaceklaskowski/spark-workshop/blob/gh-pages/slides/README.md#toc[Spark Workshops]. 10 | 11 | This collections of notes (what some may rashly call a "book") serves as the ultimate place of mine to collect all the nuts and bolts of using https://spark.apache.org[Apache Spark]. The notes aim to help me designing and developing better products with Apache Spark. It is also a viable proof of my understanding of Apache Spark. I do eventually want to reach the highest level of mastery in Apache Spark. 12 | 13 | The collection of notes serves as *the study material* for my trainings, workshops, videos and courses about Apache Spark. Follow me on twitter https://twitter.com/jaceklaskowski[@jaceklaskowski] to know it early. You will also learn about the upcoming events about Apache Spark. 14 | 15 | Expect text and code snippets from http://spark.apache.org/community.html[Spark's mailing lists], http://spark.apache.org/docs/latest/[the official documentation of Apache Spark], http://stackoverflow.com/tags/apache-spark/info[StackOverflow], blog posts, http://search.oreilly.com/?q=learning+spark[books from O'Reilly], press releases, YouTube/Vimeo videos, http://www.quora.com/Apache-Spark[Quora], https://github.com/apache/spark[the source code of Apache Spark], etc. Attribution follows. 16 | -------------------------------------------------------------------------------- /spark-streaming-PairDStreamFunctions.adoc: -------------------------------------------------------------------------------- 1 | == [[PairDStreamFunctions]] PairDStreamFunctions 2 | 3 | `PairDStreamFunctions` is a collection of operators available on link:spark-streaming-dstreams.adoc[DStream]s of `(key, value)` pairs (through an implicit conversion). 4 | 5 | .Streaming `PairDStreamFunctions` Operators 6 | [cols="1,2",options="header",width="100%"] 7 | |=== 8 | | Operator | Description 9 | | <> | 10 | |=== 11 | 12 | === [[reduceByKeyAndWindow]] `reduceByKeyAndWindow` Operators 13 | 14 | [source, scala] 15 | ---- 16 | reduceByKeyAndWindow( 17 | reduceFunc: (V, V) => V, 18 | windowDuration: Duration): DStream[(K, V)] 19 | 20 | reduceByKeyAndWindow( 21 | reduceFunc: (V, V) => V, 22 | windowDuration: Duration, 23 | slideDuration: Duration): DStream[(K, V)] 24 | 25 | reduceByKeyAndWindow( 26 | reduceFunc: (V, V) => V, 27 | windowDuration: Duration, 28 | slideDuration: Duration, 29 | numPartitions: Int): DStream[(K, V)] 30 | 31 | reduceByKeyAndWindow( 32 | reduceFunc: (V, V) => V, 33 | windowDuration: Duration, 34 | slideDuration: Duration, 35 | partitioner: Partitioner): DStream[(K, V)] 36 | 37 | reduceByKeyAndWindow( 38 | reduceFunc: (V, V) => V, 39 | invReduceFunc: (V, V) => V, 40 | windowDuration: Duration, 41 | slideDuration: Duration = self.slideDuration, 42 | numPartitions: Int = ssc.sc.defaultParallelism, 43 | filterFunc: ((K, V)) => Boolean = null): DStream[(K, V)] 44 | 45 | reduceByKeyAndWindow( 46 | reduceFunc: (V, V) => V, 47 | invReduceFunc: (V, V) => V, 48 | windowDuration: Duration, 49 | slideDuration: Duration, 50 | partitioner: Partitioner, 51 | filterFunc: ((K, V)) => Boolean): DStream[(K, V)] 52 | ---- 53 | 54 | `reduceByKeyAndWindow` returns a `ReducedWindowedDStream` with the input `reduceFunc`, `invReduceFunc` and `filterFunc` functions link:../spark-sparkcontext.adoc#clean[cleaned up]. 55 | 56 | TIP: Enable `DEBUG` logging level for `org.apache.spark.streaming.dstream.ReducedWindowedDStream` to see the times for window, slide, zero with current and previous windows in the logs. 57 | -------------------------------------------------------------------------------- /spark-streaming-windoweddstreams.adoc: -------------------------------------------------------------------------------- 1 | == WindowedDStreams 2 | 3 | `WindowedDStream` (aka *windowed stream*) is an internal link:spark-streaming-dstreams.adoc[DStream] with dependency on the `parent` stream. 4 | 5 | NOTE: It is the result of link:spark-streaming-windowedoperators.adoc#window[window operators]. 6 | 7 | `windowDuration` has to be a multiple of the parent stream's slide duration. 8 | 9 | `slideDuration` has to be a multiple of the parent stream's slide duration. 10 | 11 | NOTE: When `windowDuration` or `slideDuration` are _not_ multiples of the parent stream's slide duration, `Exception` is thrown. 12 | 13 | The parent's RDDs are automatically changed to be link:../spark-rdd-caching.adoc[persisted] at link:../spark-rdd-StorageLevel.adoc[`MEMORY_ONLY_SER` storage level] (since they need to last longer than the parent's slide duration for this stream to generate its own RDDs). 14 | 15 | Obviously, slide duration of the stream is given explicitly (and must be a multiple of the parent's slide duration). 16 | 17 | `parentRememberDuration` is extended to cover the parent's `rememberDuration` and the window duration. 18 | 19 | `compute` method always returns a RDD, either `PartitionerAwareUnionRDD` or `UnionRDD`, depending on the number of the link:../spark-rdd-Partitioner.adoc[Partitioner] defined by the RDDs in the window. It uses link:spark-streaming-windowedoperators.adoc#slice[slice] operator on the parent stream (using the slice window of `[now - windowDuration + parent.slideDuration, now]`). 20 | 21 | If only one partitioner is used across the RDDs in window, `PartitionerAwareUnionRDD` is created and you should see the following DEBUG message in the logs: 22 | 23 | ``` 24 | DEBUG WindowedDStream: Using partition aware union for windowing at [time] 25 | ``` 26 | 27 | Otherwise, when there are multiple different partitioners in use, `UnionRDD` is created and you should see the following DEBUG message in the logs: 28 | 29 | ``` 30 | DEBUG WindowedDStream: Using normal union for windowing at [time] 31 | ``` 32 | 33 | [TIP] 34 | ==== 35 | Enable `DEBUG` logging level for `org.apache.spark.streaming.dstream.WindowedDStream` logger to see what happens inside `WindowedDStream`. 36 | 37 | Add the following line to `conf/log4j.properties`: 38 | 39 | ``` 40 | log4j.logger.org.apache.spark.streaming.dstream.WindowedDStream=DEBUG 41 | ``` 42 | ==== 43 | -------------------------------------------------------------------------------- /spark-streaming-receivers.adoc: -------------------------------------------------------------------------------- 1 | == [[Receiver]] Receivers 2 | 3 | *Receivers* run on link:spark-workers.adoc[workers] to receive external data. They are created and belong to link:spark-streaming-receiverinputdstreams.adoc[ReceiverInputDStreams]. 4 | 5 | NOTE: link:spark-streaming-receivertracker.adoc[ReceiverTracker] launches a receiver on a worker. 6 | 7 | It is represented by https://github.com/apache/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala[abstract class Receiver] that is parameterized by the type of the elements it processes as well as link:../spark-rdd-StorageLevel.adoc[StorageLevel]. 8 | 9 | NOTE: You use link:spark-streaming-streamingcontext.adoc#receiverStream[StreamingContext.receiverStream] method to register a custom `Receiver` to a streaming context. 10 | 11 | The abstract `Receiver` class requires the following methods to be implemented (see <>): 12 | 13 | * `onStart()` that starts the receiver when the application starts. 14 | * `onStop()` that stops the receiver. 15 | 16 | A receiver is identified by the unique identifier `Receiver.streamId` (that corresponds to the unique identifier of the receiver input stream it is associated with). 17 | 18 | NOTE: link:../spark-rdd-StorageLevel.adoc[StorageLevel] of a receiver is used to instantiate link:spark-streaming-receivedblockhandlers.adoc[ReceivedBlockHandler] in link:spark-streaming-receiversupervisors.adoc#ReceiverSupervisorImpl[ReceiverSupervisorImpl]. 19 | 20 | A receiver uses `store` methods to store received data as data blocks into Spark's memory. 21 | 22 | NOTE: Receivers must have link:spark-streaming-receiversupervisors.adoc[ReceiverSupervisors] attached before they can be started since `store` and management methods simply pass calls on to the respective methods in the ReceiverSupervisor. 23 | 24 | A receiver can be in one of the three states: `Initialized`, `Started`, and `Stopped`. 25 | 26 | === [[custom-receiver]] Custom Receiver 27 | 28 | [source, scala] 29 | ---- 30 | import org.apache.spark.storage.StorageLevel 31 | import org.apache.spark.streaming.{Seconds, StreamingContext} 32 | import org.apache.spark.streaming.receiver.Receiver 33 | 34 | final class MyStringReceiver extends Receiver[String](StorageLevel.NONE) { 35 | 36 | def onStart() = { 37 | println("onStart called") 38 | } 39 | 40 | def onStop() = { 41 | println("onStop called") 42 | } 43 | } 44 | 45 | val ssc = new StreamingContext(sc, Seconds(5)) 46 | val strings = ssc.receiverStream(new MyStringReceiver) 47 | strings.print 48 | 49 | ssc.start 50 | 51 | // MyStringReceiver will print "onStart called" 52 | 53 | ssc.stop() 54 | 55 | // MyStringReceiver will print "onStop called" 56 | ---- 57 | -------------------------------------------------------------------------------- /spark-streaming-receivedblockhandlers.adoc: -------------------------------------------------------------------------------- 1 | == ReceivedBlockHandlers 2 | 3 | `ReceivedBlockHandler` represents how to handle the storage of blocks received by link:spark-streaming-receivers.adoc[receivers]. 4 | 5 | NOTE: It is used by link:spark-streaming-receiversupervisors.adoc#ReceiverSupervisorImpl[ReceiverSupervisorImpl] (as the internal link:spark-streaming-receiversupervisors.adoc#ReceiverSupervisorImpl-receivedBlockHandler[receivedBlockHandler]). 6 | 7 | === [[contract]] ReceivedBlockHandler Contract 8 | 9 | `ReceivedBlockHandler` is a `private[streaming] trait`. It comes with two methods: 10 | 11 | * `storeBlock(blockId: StreamBlockId, receivedBlock: ReceivedBlock): ReceivedBlockStoreResult` to store a received block as `blockId`. 12 | * `cleanupOldBlocks(threshTime: Long)` to clean up blocks older than `threshTime`. 13 | 14 | NOTE: `cleanupOldBlocks` implies that there is a relation between blocks and the time they arrived. 15 | 16 | === [[implementations]] Implementations of ReceivedBlockHandler Contract 17 | 18 | There are two implementations of `ReceivedBlockHandler` contract: 19 | 20 | * `BlockManagerBasedBlockHandler` that stores received blocks in Spark's link:spark-blockmanager.adoc[BlockManager] with the specified link:../spark-rdd-StorageLevel.adoc[StorageLevel]. 21 | + 22 | Read <> in this document. 23 | 24 | * `WriteAheadLogBasedBlockHandler` that stores received blocks in a write ahead log and Spark's link:spark-blockmanager.adoc[BlockManager]. It is a more advanced option comparing to a simpler <>. 25 | + 26 | Read <> in this document. 27 | 28 | ==== [[BlockManagerBasedBlockHandler]] BlockManagerBasedBlockHandler 29 | 30 | `BlockManagerBasedBlockHandler` is the default `ReceivedBlockHandler` in Spark Streaming. 31 | 32 | It uses link:spark-blockmanager.adoc[BlockManager] and a receiver's link:../spark-rdd-StorageLevel.adoc[StorageLevel]. 33 | 34 | `cleanupOldBlocks` is not used as blocks are cleared by _some other means_ (FIXME) 35 | 36 | `putResult` returns `BlockManagerBasedStoreResult`. It uses `BlockManager.putIterator` to store `ReceivedBlock`. 37 | 38 | ==== [[WriteAheadLogBasedBlockHandler]] WriteAheadLogBasedBlockHandler 39 | 40 | `WriteAheadLogBasedBlockHandler` is used when link:spark-streaming-settings.adoc[spark.streaming.receiver.writeAheadLog.enable] is `true`. 41 | 42 | It uses link:spark-blockmanager.adoc[BlockManager], a receiver's `streamId` and link:../spark-rdd-StorageLevel.adoc[StorageLevel], link:spark-SparkConf.adoc[SparkConf] for additional link:spark-streaming-settings.adoc[configuration settings], Hadoop Configuration, the checkpoint directory. 43 | -------------------------------------------------------------------------------- /spark-streaming-mapwithstatedstreams.adoc: -------------------------------------------------------------------------------- 1 | == MapWithStateDStream 2 | 3 | `MapWithStateDStream` is the result of link:spark-streaming-operators.adoc#mapWithState[mapWithState] stateful operator. 4 | 5 | It extends link:spark-streaming-dstreams.adoc#contract[DStream Contract] with the following additional method: 6 | 7 | [source, scala] 8 | ---- 9 | def stateSnapshots(): DStream[(KeyType, StateType)] 10 | ---- 11 | 12 | NOTE: `MapWithStateDStream` is a Scala `sealed abstract class` (and hence all the available implementations are in the source file). 13 | 14 | NOTE: <> is the only implementation of `MapWithStateDStream` (see below in this document for more coverage). 15 | 16 | === [[MapWithStateDStreamImpl]] MapWithStateDStreamImpl 17 | 18 | `MapWithStateDStreamImpl` is an internal link:spark-streaming-dstreams.adoc[DStream] with dependency on the parent `dataStream` key-value dstream. It uses a custom internal dstream called `internalStream` (of type <>). 19 | 20 | `slideDuration` is exactly the slide duration of the internal stream `internalStream`. 21 | 22 | `dependencies` returns a single-element collection with the internal stream `internalStream`. 23 | 24 | The `compute` method may or may not return a `RDD[MappedType]` by `getOrCompute` on the internal stream and...TK 25 | 26 | CAUTION: FIXME 27 | 28 | === [[InternalMapWithStateDStream]] InternalMapWithStateDStream 29 | 30 | `InternalMapWithStateDStream` is an internal dstream to support <> and uses `dataStream` (as `parent` of type `DStream[(K, V)]`) as well as `StateSpecImpl[K, V, S, E]` (as `spec`). 31 | 32 | `InternalMapWithStateDStream` is a `DStream[MapWithStateRDDRecord[K, S, E]]` that uses link:../spark-rdd-StorageLevel.adoc[`MEMORY_ONLY` storage level] by default. 33 | 34 | `InternalMapWithStateDStream` uses the ``StateSpec``'s partitioner or link:../spark-rdd-HashPartitioner.adoc[HashPartitioner] (with link:../spark-sparkcontext.adoc#defaultParallelism[SparkContext's defaultParallelism]). 35 | 36 | `slideDuration` is the slide duration of `parent`. 37 | 38 | `dependencies` is a single-element collection with the `parent` stream. 39 | 40 | It forces link:spark-streaming-checkpointing.adoc[checkpointing] (i.e. `mustCheckpoint` flag is enabled). 41 | 42 | When initialized, if link:spark-streaming-checkpointing.adoc#checkpoing-interval[checkpoint interval] is _not_ set, it sets it as ten times longer than the slide duration of the `parent` stream (the multiplier is not configurable and always `10`). 43 | 44 | Computing a `RDD[MapWithStateRDDRecord[K, S, E]]` (i.e. `compute` method) first looks up a previous RDD for the last `slideDuration`. 45 | 46 | If the RDD is found, it is returned as is given the partitioners of the RDD and the stream are equal. Otherwise, when the partitioners are different, the RDD is "repartitioned" using `MapWithStateRDD.createFromRDD`. 47 | 48 | CAUTION: FIXME `MapWithStateRDD.createFromRDD` 49 | -------------------------------------------------------------------------------- /SUMMARY.adoc: -------------------------------------------------------------------------------- 1 | = Summary 2 | 3 | . link:book-intro.adoc[Introduction] 4 | 5 | . link:spark-streaming.adoc[Spark Streaming] 6 | . link:spark-streaming-streamingcontext.adoc[StreamingContext] 7 | .. link:spark-streaming-operators.adoc[Stream Operators] 8 | .. link:spark-streaming-windowedoperators.adoc[Windowed Operators] 9 | .. link:spark-streaming-operators-saveas.adoc[SaveAs Operators] 10 | .. link:spark-streaming-operators-stateful.adoc[Stateful Operators] 11 | .. link:spark-streaming-PairDStreamFunctions.adoc[PairDStreamFunctions] 12 | 13 | . link:spark-streaming-webui.adoc[web UI and Streaming Statistics Page] 14 | . link:spark-streaming-streaminglisteners.adoc[Streaming Listeners] 15 | . link:spark-streaming-checkpointing.adoc[Checkpointing] 16 | . link:spark-streaming-jobscheduler.adoc[JobScheduler] 17 | .. link:spark-streaming-InputInfoTracker.adoc[InputInfoTracker] 18 | . link:spark-streaming-jobgenerator.adoc[JobGenerator] 19 | . link:spark-streaming-dstreamgraph.adoc[DStreamGraph] 20 | 21 | . link:spark-streaming-dstreams.adoc[Discretized Streams (DStreams)] 22 | .. link:spark-streaming-inputdstreams.adoc[Input DStreams] 23 | .. link:spark-streaming-receiverinputdstreams.adoc[ReceiverInputDStreams] 24 | .. link:spark-streaming-constantinputdstreams.adoc[ConstantInputDStreams] 25 | .. link:spark-streaming-foreachdstreams.adoc[ForEachDStreams] 26 | .. link:spark-streaming-windoweddstreams.adoc[WindowedDStreams] 27 | .. link:spark-streaming-mapwithstatedstreams.adoc[MapWithStateDStreams] 28 | .. link:spark-streaming-statedstreams.adoc[StateDStreams] 29 | .. link:spark-streaming-transformeddstreams.adoc[TransformedDStream] 30 | 31 | . link:spark-streaming-receivers.adoc[Receivers] 32 | .. link:spark-streaming-receivertracker.adoc[ReceiverTracker] 33 | .. link:spark-streaming-receiversupervisors.adoc[ReceiverSupervisors] 34 | .. link:spark-streaming-receivedblockhandlers.adoc[ReceivedBlockHandlers] 35 | 36 | . link:spark-streaming-kafka.adoc[Ingesting Data from Kafka] 37 | .. link:spark-streaming-kafka-KafkaUtils.adoc[KafkaUtils -- Creating Kafka DStreams and RDDs] 38 | .. link:spark-streaming-kafka-DirectKafkaInputDStream.adoc[DirectKafkaInputDStream -- Direct Kafka DStream] 39 | .. link:spark-streaming-kafka-ConsumerStrategy.adoc[ConsumerStrategy -- Kafka Consumers' Post-Configuration API] 40 | ... link:spark-streaming-kafka-ConsumerStrategies.adoc[ConsumerStrategies Factory Object] 41 | .. link:spark-streaming-kafka-LocationStrategy.adoc[LocationStrategy -- Preferred Hosts per Topic Partitions] 42 | .. link:spark-streaming-kafka-KafkaRDD.adoc[KafkaRDD] 43 | .. link:spark-streaming-kafka-HasOffsetRanges.adoc[HasOffsetRanges and OffsetRange] 44 | 45 | . link:spark-streaming-recurringtimer.adoc[RecurringTimer] 46 | . link:spark-streaming-backpressure.adoc[Backpressure] 47 | . link:spark-streaming-dynamic-allocation.adoc[Dynamic Allocation (Elastic Scaling)] 48 | .. link:spark-streaming-ExecutorAllocationManager.adoc[ExecutorAllocationManager] 49 | . link:spark-streaming-StreamingSource.adoc[StreamingSource] 50 | . link:spark-streaming-settings.adoc[Settings] 51 | -------------------------------------------------------------------------------- /spark-streaming-kafka-HasOffsetRanges.adoc: -------------------------------------------------------------------------------- 1 | == HasOffsetRanges and OffsetRange 2 | 3 | === [[HasOffsetRanges]] HasOffsetRanges 4 | 5 | `HasOffsetRanges` represents an object that has a collection of <>s (i.e. a range of offsets from a single Kafka topic partition). 6 | 7 | `HasOffsetRanges` is part of `org.apache.spark.streaming.kafka010` package. 8 | 9 | NOTE: link:spark-streaming-kafka-KafkaRDD.adoc[KafkaRDD] is a `HasOffsetRanges` object. 10 | 11 | You can access `HasOffsetRanges` given a link:spark-streaming-kafka-KafkaRDD.adoc[KafkaRDD] as follows: 12 | 13 | [source, scala] 14 | ---- 15 | import org.apache.spark.streaming.kafka010.KafkaUtils 16 | KafkaUtils.createDirectStream(...).foreachRDD { rdd => 17 | import org.apache.spark.streaming.kafka010.OffsetRange 18 | val offsetRanges: Array[OffsetRange] = rdd.asInstanceOf[HasOffsetRanges].offsetRanges 19 | } 20 | ---- 21 | 22 | === [[OffsetRange]] OffsetRange 23 | 24 | `OffsetRange` represents a range of offsets from a single Kafka https://kafka.apache.org/0100/javadoc/org/apache/kafka/common/TopicPartition.html[TopicPartition] (i.e. a topic name and partition number). 25 | 26 | `OffsetRange` holds a `topic`, `partition` number, `fromOffset` (inclusive) and `untilOffset` (exclusive) offsets. 27 | 28 | You can create instances of `OffsetRange` using the <>. You can then count the number of records in a topic partition using <> method. 29 | 30 | [source, scala] 31 | ---- 32 | // Start spark-shell with spark-streaming-kafka-0-10_2.11 dependency 33 | // --packages org.apache.spark:spark-streaming-kafka-0-10_2.11:2.1.0-SNAPSHOT 34 | import org.apache.spark.streaming.kafka010.OffsetRange 35 | 36 | scala> val offsets = OffsetRange(topic = "spark-logs", partition = 0, fromOffset = 2, untilOffset = 5) 37 | offsets: org.apache.spark.streaming.kafka010.OffsetRange = OffsetRange(topic: 'spark-logs', partition: 0, range: [2 -> 5]) 38 | 39 | scala> offsets.count 40 | res0: Long = 3 41 | 42 | scala> offsets.topicPartition 43 | res1: org.apache.kafka.common.TopicPartition = spark-logs-0 44 | ---- 45 | 46 | `OffsetRange` is part of `org.apache.spark.streaming.kafka010` package. 47 | 48 | ==== [[OffsetRange-creating-instance]] Creating OffsetRange Instance 49 | 50 | You can create instances of `OffsetRange` using the following factory methods (from `OffsetRange` companion object): 51 | 52 | [source, scala] 53 | ---- 54 | OffsetRange.create( 55 | topic: String, 56 | partition: Int, 57 | fromOffset: Long, 58 | untilOffset: Long): OffsetRange 59 | 60 | OffsetRange.create( 61 | topicPartition: TopicPartition, 62 | fromOffset: Long, 63 | untilOffset: Long): OffsetRange 64 | 65 | OffsetRange.apply( 66 | topic: String, 67 | partition: Int, 68 | fromOffset: Long, 69 | untilOffset: Long): OffsetRange 70 | 71 | OffsetRange.apply( 72 | topicPartition: TopicPartition, 73 | fromOffset: Long, 74 | untilOffset: Long): OffsetRange 75 | ---- 76 | 77 | ==== [[OffsetRange-count]] Counting Records in Topic Partition -- `count` method 78 | 79 | [source, scala] 80 | ---- 81 | count(): Long 82 | ---- 83 | 84 | `count` counts the number of records in a `OffsetRange`. 85 | -------------------------------------------------------------------------------- /spark-streaming-inputdstreams.adoc: -------------------------------------------------------------------------------- 1 | == [[InputDStream]] Input DStreams 2 | 3 | *Input DStreams* in Spark Streaming are the way to ingest data from external data sources. They are represented as `InputDStream` abstract class. 4 | 5 | === [[contract]] InputDStream Contract 6 | 7 | `InputDStream` is the abstract base class for all input link:spark-streaming-dstreams.adoc[DStreams]. It provides two abstract methods `start()` and `stop()` to start and stop ingesting data, respectively. 8 | 9 | When instantiated, an `InputDStream` registers itself as an input stream (using link:spark-streaming-dstreamgraph.adoc[DStreamGraph.addInputStream]) and, while doing so, is told about its owning link:spark-streaming-dstreamgraph.adoc[DStreamGraph]. 10 | 11 | It asks for its own unique identifier using `StreamingContext.getNewInputStreamId()`. 12 | 13 | NOTE: It is StreamingContext to maintain the identifiers and how many input streams have already been created. 14 | 15 | `InputDStream` has a human-readable `name` that is made up from a nicely-formatted part based on the class name and the unique identifier. 16 | 17 | TIP: Name your custom `InputDStream` using the CamelCase notation with the suffix *InputDStream*, e.g. MyCustomInputDStream. 18 | 19 | * `slideDuration` calls link:spark-streaming-dstreamgraph.adoc[DStreamGraph.batchDuration]. 20 | 21 | * `dependencies` method returns an empty collection. 22 | 23 | NOTE: `compute(validTime: Time): Option[RDD[T]]` abstract method from link:spark-streaming-dstreams.adoc[DStream] abstract class is not defined. 24 | 25 | Custom implementations of `InputDStream` can override (and actually provide!) the optional link:spark-streaming-backpressure.adoc#RateController[RateController]. It is undefined by default. 26 | 27 | === Custom Input DStream 28 | 29 | Here is an example of a custom input dstream that produces an RDD out of the input collection of elements (of type `T`). 30 | 31 | NOTE: It is similar to link:spark-streaming-constantinputdstreams.adoc[ConstantInputDStreams], but this custom implementation does not use an external RDD, but generates its own. 32 | 33 | [source, scala] 34 | ---- 35 | package pl.japila.spark.streaming 36 | 37 | import org.apache.spark.rdd.RDD 38 | import org.apache.spark.streaming.{ Time, StreamingContext } 39 | import org.apache.spark.streaming.dstream.InputDStream 40 | 41 | import scala.reflect.ClassTag 42 | 43 | class CustomInputDStream[T: ClassTag](ssc: StreamingContext, seq: Seq[T]) 44 | extends InputDStream[T](ssc) { 45 | override def compute(validTime: Time): Option[RDD[T]] = { 46 | Some(ssc.sparkContext.parallelize(seq)) 47 | } 48 | override def start(): Unit = {} 49 | override def stop(): Unit = {} 50 | } 51 | ---- 52 | 53 | Its use could be as simple as follows (compare it to the link:spark-streaming-constantinputdstreams.adoc#example[example of ConstantInputDStreams]): 54 | 55 | [source, scala] 56 | ---- 57 | // sc is the SparkContext instance 58 | import org.apache.spark.streaming.Seconds 59 | val ssc = new StreamingContext(sc, batchDuration = Seconds(5)) 60 | 61 | // Create the collection of numbers 62 | val nums = 0 to 9 63 | 64 | // Create constant input dstream with the RDD 65 | import pl.japila.spark.streaming.CustomInputDStream 66 | val cis = new CustomInputDStream(ssc, nums) 67 | 68 | // Sample stream computation 69 | cis.print 70 | ---- 71 | 72 | TIP: Copy and paste it to `spark-shell` to run it. 73 | -------------------------------------------------------------------------------- /spark-streaming-streaminglisteners.adoc: -------------------------------------------------------------------------------- 1 | == [[StreamingListener]] Streaming Listeners 2 | 3 | *Streaming listeners* are Spark listeners interested in <> like batch submitted, started or completed. 4 | 5 | Streaming listeners implement http://spark.apache.org/docs/latest/api/scala/#org.apache.spark.streaming.scheduler.StreamingListener[org.apache.spark.streaming.scheduler.StreamingListener] listener interface and process http://spark.apache.org/docs/latest/api/scala/#org.apache.spark.streaming.scheduler.StreamingListenerEvent[StreamingListenerEvent] events. 6 | 7 | The following streaming listeners are available in Spark Streaming: 8 | 9 | * <> 10 | * link:spark-streaming-backpressure.adoc#RateController[RateController] 11 | 12 | === [[StreamingListenerEvent]] StreamingListenerEvent Events 13 | 14 | * `StreamingListenerBatchSubmitted` is posted when link:spark-streaming-jobscheduler.adoc#submitJobSet[streaming jobs are submitted for execution] and triggers `StreamingListener.onBatchSubmitted` (see <>). 15 | 16 | * `StreamingListenerBatchStarted` triggers `StreamingListener.onBatchStarted` 17 | 18 | * `StreamingListenerBatchCompleted` is posted to inform that link:spark-streaming-jobscheduler.adoc#JobCompleted[a collection of streaming jobs has completed], i.e. all the streaming jobs in link:spark-streaming-jobscheduler.adoc#JobSet[JobSet] have stopped their execution. 19 | 20 | === [[StreamingJobProgressListener]] StreamingJobProgressListener 21 | 22 | `StreamingJobProgressListener` is a streaming listener that collects information for link:spark-streaming-StreamingSource.adoc[StreamingSource] and link:spark-streaming-webui.adoc[Streaming page in web UI]. 23 | 24 | NOTE: A `StreamingJobProgressListener` is created while link:spark-streaming-streamingcontext.adoc#creating-instance[`StreamingContext` is created] and later registered as a `StreamingListener` and link:../spark-SparkListener.adoc[SparkListener] when link:spark-streaming-webui.adoc[Streaming tab] is created. 25 | 26 | ==== [[StreamingJobProgressListener-onBatchSubmitted]] onBatchSubmitted 27 | 28 | For `StreamingListenerBatchSubmitted(batchInfo: BatchInfo)` events, it stores `batchInfo` batch information in the internal `waitingBatchUIData` registry per batch time. 29 | 30 | The number of entries in `waitingBatchUIData` registry contributes to `numUnprocessedBatches` (together with `runningBatchUIData`), `waitingBatches`, and `retainedBatches`. It is also used to look up the batch data for a batch time (in `getBatchUIData`). 31 | 32 | `numUnprocessedBatches`, `waitingBatches` are used in link:spark-streaming-StreamingSource.adoc[StreamingSource]. 33 | 34 | NOTE: `waitingBatches` and `runningBatches` are displayed together in link:spark-streaming-webui.adoc#active-batches[Active Batches in Streaming tab in web UI]. 35 | 36 | ==== onBatchStarted 37 | 38 | CAUTION: FIXME 39 | 40 | ==== [[onBatchCompleted]] onBatchCompleted 41 | 42 | CAUTION: FIXME 43 | 44 | ==== [[retainedBatches]] Retained Batches 45 | 46 | `retainedBatches` are waiting, running, and completed batches that link:spark-streaming-webui.adoc[web UI uses to display streaming statistics]. 47 | 48 | The number of retained batches is controlled by link:spark-streaming-settings.adoc[spark.streaming.ui.retainedBatches]. 49 | -------------------------------------------------------------------------------- /spark-streaming-receiverinputdstreams.adoc: -------------------------------------------------------------------------------- 1 | == ReceiverInputDStreams - Input Streams with Receivers 2 | 3 | *Receiver Input Streams* (`ReceiverInputDStreams`) are specialized link:spark-streaming-inputdstreams.adoc[input streams] that use link:spark-streaming-receivers.adoc[receivers] to receive data (and hence the name which stands for an `InputDStream` with a receiver). 4 | 5 | NOTE: Receiver input streams run receivers as long-running tasks that occupy a core per stream. 6 | 7 | `ReceiverInputDStream` abstract class defines the following abstract method that custom implementations use to create receivers: 8 | 9 | ``` 10 | def getReceiver(): Receiver[T] 11 | ``` 12 | 13 | The receiver is then sent to and run on workers (when link:spark-streaming-receivertracker.adoc#starting[ReceiverTracker is started]). 14 | 15 | [NOTE] 16 | ==== 17 | A fine example of a very minimalistic yet still useful implementation of `ReceiverInputDStream` class is the pluggable input stream `org.apache.spark.streaming.dstream.PluggableInputDStream` (https://github.com/apache/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala[the sources on GitHub]). It requires a `Receiver` to be given (by a developer) and simply returns it in `getReceiver`. 18 | 19 | `PluggableInputDStream` is used by link:spark-streaming-streamingcontext.adoc#creating-receivers[StreamingContext.receiverStream()] method. 20 | ==== 21 | 22 | `ReceiverInputDStream` uses `ReceiverRateController` when link:spark-streaming-settings.adoc[spark.streaming.backpressure.enabled] is enabled. 23 | 24 | [NOTE] 25 | ==== 26 | Both, `start()` and `stop` methods are implemented in `ReceiverInputDStream`, but do nothing. `ReceiverInputDStream` management is left to link:spark-streaming-receivertracker.adoc[ReceiverTracker]. 27 | 28 | Read link:spark-streaming-receivertracker.adoc#ReceiverTrackerEndpoint-startReceiver[ReceiverTrackerEndpoint.startReceiver] for more details. 29 | ==== 30 | 31 | The source code of `ReceiverInputDStream` is https://github.com/apache/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala[here at GitHub]. 32 | 33 | === [[compute]] Generate RDDs for Batch Interval -- `compute` Method 34 | 35 | The abstract `compute(validTime: Time): Option[RDD[T]]` method (from link:spark-streaming-dstreams.adoc[DStream]) uses link:spark-streaming-dstreamgraph.adoc[start time of DStreamGraph], i.e. the start time of StreamingContext, to check whether `validTime` input parameter is really valid. 36 | 37 | If the time to generate RDDs (`validTime`) is earlier than the start time of StreamingContext, an empty `BlockRDD` is generated. 38 | 39 | Otherwise, link:spark-streaming-receivertracker.adoc[ReceiverTracker] is requested for all the blocks that have been allocated to this stream for this batch (using `ReceiverTracker.getBlocksOfBatch`). 40 | 41 | The number of records received for the batch for the input stream (as link:spark-streaming-InputInfoTracker.adoc#StreamInputInfo[StreamInputInfo]) is registered with link:spark-streaming-InputInfoTracker.adoc#reportInfo[InputInfoTracker]. 42 | 43 | If all link:spark-blockdatamanager.adoc[BlockIds] have `WriteAheadLogRecordHandle`, a `WriteAheadLogBackedBlockRDD` is generated. Otherwise, a `BlockRDD` is. 44 | 45 | === [[back-pressure]] Back Pressure 46 | 47 | CAUTION: FIXME 48 | 49 | link:spark-streaming-backpressure.adoc[Back pressure] for input dstreams with receivers can be configured using link:spark-streaming-settings.adoc#back-pressure[spark.streaming.backpressure.enabled] setting. 50 | 51 | NOTE: Back pressure is disabled by default. 52 | -------------------------------------------------------------------------------- /spark-streaming-settings.adoc: -------------------------------------------------------------------------------- 1 | == Settings 2 | 3 | The following list are the settings used to configure Spark Streaming applications. 4 | 5 | CAUTION: FIXME Describe how to set them in streaming applications. 6 | 7 | * `spark.streaming.kafka.maxRetries` (default: `1`) sets up the number of connection attempts to Kafka brokers. 8 | 9 | * `spark.streaming.receiver.writeAheadLog.enable` (default: `false`) controls what link:spark-streaming-receivedblockhandlers.adoc[ReceivedBlockHandler] to use: `WriteAheadLogBasedBlockHandler` or `BlockManagerBasedBlockHandler`. 10 | 11 | * `spark.streaming.receiver.blockStoreTimeout` (default: `30`) time in seconds to wait until both writes to a write-ahead log and BlockManager complete successfully. 12 | 13 | * `spark.streaming.clock` (default: `org.apache.spark.util.SystemClock`) specifies a fully-qualified class name that extends `org.apache.spark.util.Clock` to represent time. It is used in link:spark-streaming-jobgenerator.adoc[JobGenerator]. 14 | 15 | * `spark.streaming.ui.retainedBatches` (default: `1000`) controls the number of `BatchUIData` elements about completed batches in a first-in-first-out (FIFO) queue that are used to link:spark-streaming-webui.adoc[display statistics in Streaming page in web UI]. 16 | * `spark.streaming.receiverRestartDelay` (default: `2000`) - the time interval between a receiver is stopped and started again. 17 | 18 | * `spark.streaming.concurrentJobs` (default: `1`) is the number of concurrent jobs, i.e. threads in link:spark-streaming-jobscheduler.adoc#streaming-job-executor[streaming-job-executor thread pool]. 19 | 20 | * `spark.streaming.stopSparkContextByDefault` (default: `true`) controls whether (`true`) or not (`false`) to stop the underlying SparkContext (regardless of whether this `StreamingContext` has been started). 21 | 22 | [[spark_streaming_kafka_maxRatePerPartition]] 23 | * `spark.streaming.kafka.maxRatePerPartition` (default: `0`) if non-`0` sets maximum number of messages per partition. 24 | 25 | * `spark.streaming.manualClock.jump` (default: `0`) offsets (aka _jumps_) the system time, i.e. adds its value to checkpoint time, when used with the clock being a subclass of `org.apache.spark.util.ManualClock`. It is used when link:spark-streaming-jobgenerator.adoc[JobGenerator] is restarted from checkpoint. 26 | 27 | * `spark.streaming.unpersist` (default: `true`) is a flag to control whether link:spark-streaming-dstreams.adoc#clearMetadata[output streams should unpersist old RDDs]. 28 | 29 | * `spark.streaming.gracefulStopTimeout` (default: 10 * link:spark-streaming-dstreamgraph.adoc#batch-interval[batch interval]) 30 | 31 | * `spark.streaming.stopGracefullyOnShutdown` (default: `false`) controls link:spark-streaming-streamingcontext.adoc#stop[whether to stop StreamingContext gracefully or not] and is used by link:spark-streaming-streamingcontext.adoc#stopOnShutdown[stopOnShutdown Shutdown Hook]. 32 | 33 | === [[checkpointing]] Checkpointing 34 | 35 | * `spark.streaming.checkpoint.directory` - when set and link:spark-streaming-streamingcontext.adoc#creating-instance[StreamingContext is created], the value of the setting gets passed on to link:spark-streaming-streamingcontext.adoc#checkpoint[StreamingContext.checkpoint] method. 36 | 37 | === [[back-pressure]] Back Pressure 38 | 39 | * `spark.streaming.backpressure.enabled` (default: `false`) - enables (`true`) or disables (`false`) link:spark-streaming-backpressure.adoc[back pressure] in link:spark-streaming-receiverinputdstreams.adoc#back-pressure[input streams with receivers] or link:spark-streaming-kafka-DirectKafkaInputDStream.adoc#back-pressure[DirectKafkaInputDStream]. 40 | 41 | * `spark.streaming.backpressure.rateEstimator` (default: `pid`) is the link:spark-streaming-backpressure.adoc#RateEstimator[RateEstimator] to use. 42 | -------------------------------------------------------------------------------- /spark-streaming.adoc: -------------------------------------------------------------------------------- 1 | == Spark Streaming -- Streaming RDDs 2 | 3 | *Spark Streaming* is the incremental *micro-batching stream processing framework* for Spark. 4 | 5 | Spark Streaming offers the data abstraction called link:spark-streaming-dstreams.adoc[DStream] that hides the complexity of dealing with a continuous data stream and makes it as easy for programmers as using one single RDD at a time. 6 | 7 | That is why Spark Streaming is also called a *micro-batching streaming framework* as a batch is one RDD at a time. 8 | 9 | NOTE: I think Spark Streaming shines on performing the *T* stage well, i.e. the transformation stage, while leaving the *E* and *L* stages for more specialized tools like link:spark-streaming-kafka.adoc[Apache Kafka] or frameworks like Akka. 10 | 11 | For a software developer, a `DStream` is similar to work with as a `RDD` with the DStream API to match RDD API. Interestingly, you can reuse your RDD-based code and apply it to `DStream` - a stream of RDDs - with no changes at all (through link:spark-streaming-operators.adoc#foreachRDD[foreachRDD]). 12 | 13 | It runs <> every <> to pull and process data (often called _records_) from one or many link:spark-streaming-inputdstreams.adoc[input streams]. 14 | 15 | Each batch link:spark-streaming-dstreams.adoc#contract[computes] (_generates_) a RDD for data in input streams for a given batch and link:spark-streaming-jobgenerator.adoc#generateJobs[submits a Spark job to compute the result]. It does this over and over again until link:spark-streaming-streamingcontext.adoc#stopping[the streaming context is stopped] (and the owning streaming application terminated). 16 | 17 | To avoid losing records in case of failure, Spark Streaming supports link:spark-streaming-checkpointing.adoc[checkpointing that writes received records to a highly-available HDFS-compatible storage] and allows to recover from temporary downtimes. 18 | 19 | Spark Streaming allows for integration with real-time data sources ranging from such basic ones like a HDFS-compatible file system or socket connection to more advanced ones like Apache Kafka or Apache Flume. 20 | 21 | Checkpointing is also the foundation of link:spark-streaming-operators-stateful.adoc[stateful] and link:spark-streaming-windowedoperators.adoc[windowed] operations. 22 | 23 | http://spark.apache.org/docs/latest/streaming-programming-guide.html#overview[About Spark Streaming from the official documentation] (that pretty much nails what it offers): 24 | 25 | > Spark Streaming is an extension of the core Spark API that enables scalable, high-throughput, fault-tolerant stream processing of live data streams. Data can be ingested from many sources like Kafka, Flume, Twitter, ZeroMQ, Kinesis, or TCP sockets, and can be processed using complex algorithms expressed with high-level functions like map, reduce, join and window. Finally, processed data can be pushed out to filesystems, databases, and live dashboards. In fact, you can apply Spark’s machine learning and graph processing algorithms on data streams. 26 | 27 | Essential concepts in Spark Streaming: 28 | 29 | * link:spark-streaming-streamingcontext.adoc[StreamingContext] 30 | * link:spark-streaming-operators.adoc[Stream Operators] 31 | * <>, Batch time, and link:spark-streaming-jobscheduler.adoc#JobSet[JobSet] 32 | * <> 33 | * link:spark-streaming-dstreams.adoc[Discretized Streams (DStreams)] 34 | * link:spark-streaming-receivers.adoc[Receivers] 35 | 36 | Other concepts often used in Spark Streaming: 37 | 38 | * *ingestion* = the act of processing streaming data. 39 | 40 | === [[batch]][[micro-batch]] Micro Batch 41 | 42 | *Micro Batch* is a collection of input records as collected by Spark Streaming that is later represented as an RDD. 43 | 44 | A *batch* is internally represented as a link:spark-streaming-jobscheduler.adoc#JobSet[JobSet]. 45 | 46 | === [[batchDuration]][[batch-interval]] Batch Interval (aka batchDuration) 47 | 48 | *Batch Interval* is a property of a Streaming application that describes how often an RDD of input records is generated. It is the time to collect input records before they become a <>. 49 | 50 | === [[Job]] Streaming Job 51 | 52 | A streaming `Job` represents a Spark computation with one or many Spark jobs. 53 | 54 | It is identified (in the logs) as `streaming job [time].[outputOpId]` with `outputOpId` being the position in the sequence of jobs in a link:spark-streaming-jobscheduler.adoc#JobSet[JobSet]. 55 | 56 | When executed, it runs the computation (the input `func` function). 57 | 58 | NOTE: A collection of streaming jobs is generated for a batch using link:spark-streaming-dstreamgraph.adoc#generateJobs[DStreamGraph.generateJobs(time: Time)]. 59 | 60 | === [[internal-registries]] Internal Registries 61 | 62 | * `nextInputStreamId` - the current InputStream id 63 | -------------------------------------------------------------------------------- /spark-streaming-windowedoperators.adoc: -------------------------------------------------------------------------------- 1 | == Windowed Operators 2 | 3 | [NOTE] 4 | ==== 5 | Go to http://spark.apache.org/docs/latest/streaming-programming-guide.html#window-operations[Window Operations] to read the official documentation. 6 | 7 | This document aims at presenting the _internals_ of window operators with examples. 8 | ==== 9 | 10 | In short, *windowed operators* allow you to apply transformations over a *sliding window* of data, i.e. build a _stateful computation_ across multiple batches. 11 | 12 | NOTE: Windowed operators, windowed operations, and window-based operations are all the same concept. 13 | 14 | By default, you apply transformations using different link:spark-streaming-operators.adoc[stream operators] to a single RDD that represents a dataset that has been built out of data received from one or many link:spark-streaming-inputdstreams.adoc[input streams]. The transformations know nothing about the past (datasets received and already processed). The computations are hence _stateless_. 15 | 16 | You can however build datasets based upon the past ones, and that is when windowed operators enter the stage. Using them allows you to cross the boundary of a single dataset (per batch) and have a series of datasets in your hands (as if the data they hold arrived in a single batch interval). 17 | 18 | .Streaming Windowed Operators 19 | [cols="1,2",options="header",width="100%"] 20 | |=== 21 | | Operator | Description 22 | | <> | 23 | | <> | 24 | | <> | 25 | |=== 26 | 27 | === [[slice]] `slice` Operators 28 | 29 | [source,scala] 30 | ---- 31 | slice(interval: Interval): Seq[RDD[T]] 32 | slice(fromTime: Time, toTime: Time): Seq[RDD[T]] 33 | ---- 34 | 35 | `slice` operators return a collection of RDDs that were generated during time interval inclusive, given as `Interval` or a pair of `Time` ends. 36 | 37 | Both `Time` ends have to be a multiple of this stream's slide duration. Otherwise, they are aligned using `Time.floor` method. 38 | 39 | When used, you should see the following INFO message in the logs: 40 | 41 | ``` 42 | INFO Slicing from [fromTime] to [toTime] (aligned to [alignedFromTime] and [alignedToTime]) 43 | ``` 44 | 45 | For every batch in the slicing interval, link:spark-streaming-dstreams.adoc#getOrCompute[a RDD is computed]. 46 | 47 | === [[window]] `window` Operators 48 | 49 | [source,scala] 50 | ---- 51 | window(windowDuration: Duration): DStream[T] 52 | window(windowDuration: Duration, slideDuration: Duration): DStream[T] 53 | ---- 54 | 55 | `window` operator creates a new stream that generates RDDs containing all the elements received during `windowDuration` with `slideDuration` link:spark-streaming-dstreams.adoc#contract[slide duration]. 56 | 57 | NOTE: `windowDuration` must be a multiple of the slide duration of the source stream. 58 | 59 | `window(windowDuration: Duration): DStream[T]` operator uses `window(windowDuration: Duration, slideDuration: Duration)` with the source stream's link:spark-streaming-dstreams.adoc#contract[slide duration]. 60 | 61 | ``` 62 | messages.window(Seconds(10)) 63 | ``` 64 | 65 | It creates link:spark-streaming-windoweddstreams.adoc[WindowedDStream] stream and register it as an output stream. 66 | 67 | NOTE: `window` operator is used by `reduceByWindow`, link:spark-streaming-PairDStreamFunctions.adoc#reduceByKeyAndWindow[reduceByKeyAndWindow] and `groupByKeyAndWindow` operators. 68 | 69 | === [[reduceByWindow]] `reduceByWindow` Operator 70 | 71 | [source, scala] 72 | ---- 73 | reduceByWindow( 74 | reduceFunc: (T, T) => T, 75 | windowDuration: Duration, 76 | slideDuration: Duration): DStream[T] 77 | 78 | reduceByWindow( 79 | reduceFunc: (T, T) => T, 80 | invReduceFunc: (T, T) => T, 81 | windowDuration: Duration, 82 | slideDuration: Duration): DStream[T] 83 | ---- 84 | 85 | `reduceByWindow` creates a new stream of RDDs of one element only that was computed using `reduceFunc` function over the data received during batch duration that later was _again_ applied to a collection of the reduced elements from the past being window duration `windowDuration` sliding `slideDuration` forward. 86 | 87 | Internally, `reduceByWindow` is exactly link:spark-streaming-dstreams.adoc#reduce[reduce] operator (with `reduceFunc`) followed by <> (of `windowDuration` and `slideDuration`) that ultimately gets ``reduce``d (again) with `reduceFunc`. 88 | 89 | [source, scala] 90 | ---- 91 | // batchDuration = Seconds(5) 92 | 93 | val clicks: InputDStream[(String, String)] = messages 94 | type T = (String, String) 95 | val reduceFn: (T, T) => T = { 96 | case in @ ((k1, v1), (k2, v2)) => 97 | println(s">>> input: $in") 98 | (k2, s"$v1 + $v2") 99 | } 100 | val windowedClicks: DStream[(String, String)] = 101 | clicks.reduceByWindow(reduceFn, windowDuration = Seconds(10), slideDuration = Seconds(5)) 102 | 103 | windowedClicks.print 104 | ---- 105 | -------------------------------------------------------------------------------- /spark-streaming-kafka-KafkaUtils.adoc: -------------------------------------------------------------------------------- 1 | == [[KafkaUtils]] `KafkaUtils` -- Creating Kafka DStreams and RDDs 2 | 3 | `KafkaUtils` is the object with the factory methods to create <> and <> from records in topics in http://kafka.apache.org[Apache Kafka]. 4 | 5 | [source, scala] 6 | ---- 7 | import org.apache.spark.streaming.kafka010.KafkaUtils 8 | ---- 9 | 10 | TIP: Use link:spark-streaming-kafka.adoc#spark-streaming-kafka-0-10[`spark-streaming-kafka-0-10` Library Dependency]. 11 | 12 | [[logging]] 13 | [TIP] 14 | ==== 15 | Enable `WARN` logging level for `org.apache.spark.streaming.kafka010.KafkaUtils` logger to see what happens inside. 16 | 17 | Add the following line to `conf/log4j.properties`: 18 | 19 | ``` 20 | log4j.logger.org.apache.spark.streaming.kafka010.KafkaUtils=WARN 21 | ``` 22 | 23 | Refer to link:../spark-logging.adoc[Logging]. 24 | ==== 25 | 26 | === [[createDirectStream]] Creating Kafka DStream -- `createDirectStream` Method 27 | 28 | [source, scala] 29 | ---- 30 | createDirectStream[K, V]( 31 | ssc: StreamingContext, 32 | locationStrategy: LocationStrategy, 33 | consumerStrategy: ConsumerStrategy[K, V]): InputDStream[ConsumerRecord[K, V]] 34 | ---- 35 | 36 | `createDirectStream` is a method that creates a link:spark-streaming-kafka-DirectKafkaInputDStream.adoc[DirectKafkaInputDStream] from a link:spark-streaming-streamingcontext.adoc[StreamingContext], link:spark-streaming-kafka-LocationStrategy.adoc[LocationStrategy], and link:spark-streaming-kafka-ConsumerStrategy.adoc[ConsumerStrategy]. 37 | 38 | [TIP] 39 | ==== 40 | Enable `DEBUG` logging level for `org.apache.kafka.clients.consumer.KafkaConsumer` logger to see what happens inside the link:spark-streaming-kafka-DirectKafkaInputDStream.adoc#consumer[Kafka consumer] that is used to communicate with Kafka broker(s). 41 | 42 | The following DEBUGs are from when a link:spark-streaming-kafka-DirectKafkaInputDStream.adoc#start[`DirectKafkaInputDStream` is started]. 43 | 44 | ``` 45 | DEBUG KafkaConsumer: Starting the Kafka consumer 46 | DEBUG KafkaConsumer: Kafka consumer created 47 | DEBUG KafkaConsumer: Subscribed to topic(s): basic1, basic2, basic3 48 | ``` 49 | 50 | Add the following line to `conf/log4j.properties`: 51 | 52 | ``` 53 | log4j.logger.org.apache.kafka.clients.consumer.KafkaConsumer=DEBUG 54 | ``` 55 | 56 | Refer to link:../spark-logging.adoc[Logging]. 57 | ==== 58 | 59 | .Using KafkaUtils.createDirectStream to Connect to Kafka Brokers 60 | [source, scala] 61 | ---- 62 | // Include org.apache.spark:spark-streaming-kafka-0-10_2.11:2.1.0-SNAPSHOT dependency in the CLASSPATH, e.g. 63 | // $ ./bin/spark-shell --packages org.apache.spark:spark-streaming-kafka-0-10_2.11:2.1.0-SNAPSHOT 64 | 65 | import org.apache.spark.streaming._ 66 | import org.apache.spark.SparkContext 67 | val sc = SparkContext.getOrCreate 68 | val ssc = new StreamingContext(sc, Seconds(5)) 69 | 70 | import org.apache.spark.streaming.kafka010._ 71 | 72 | val preferredHosts = LocationStrategies.PreferConsistent 73 | val topics = List("topic1", "topic2", "topic3") 74 | import org.apache.kafka.common.serialization.StringDeserializer 75 | val kafkaParams = Map( 76 | "bootstrap.servers" -> "localhost:9092", 77 | "key.deserializer" -> classOf[StringDeserializer], 78 | "value.deserializer" -> classOf[StringDeserializer], 79 | "group.id" -> "spark-streaming-notes", 80 | "auto.offset.reset" -> "earliest" 81 | ) 82 | import org.apache.kafka.common.TopicPartition 83 | val offsets = Map(new TopicPartition("topic3", 0) -> 2L) 84 | 85 | val dstream = KafkaUtils.createDirectStream[String, String]( 86 | ssc, 87 | preferredHosts, 88 | ConsumerStrategies.Subscribe[String, String](topics, kafkaParams, offsets)) 89 | 90 | dstream.foreachRDD { rdd => 91 | // Get the offset ranges in the RDD 92 | val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges 93 | for (o <- offsetRanges) { 94 | println(s"${o.topic} ${o.partition} offsets: ${o.fromOffset} to ${o.untilOffset}") 95 | } 96 | } 97 | 98 | ssc.start 99 | 100 | // the above code is printing out topic details every 5 seconds 101 | // until you stop it. 102 | 103 | ssc.stop(stopSparkContext = false) 104 | ---- 105 | 106 | === [[createRDD]] Creating Kafka RDD -- `createRDD` Method 107 | 108 | [source, scala] 109 | ---- 110 | def createRDD[K, V]( 111 | sc: SparkContext, 112 | kafkaParams: java.util.Map[String, Object], 113 | offsetRanges: Array[OffsetRange], 114 | locationStrategy: LocationStrategy): RDD[ConsumerRecord[K, V]] 115 | ---- 116 | 117 | `createRDD` creates a link:spark-streaming-kafka-KafkaRDD.adoc[KafkaRDD]. 118 | 119 | CAUTION: FIXME 120 | 121 | === [[fixKafkaParams]] `fixKafkaParams` Internal Method 122 | 123 | [source, scala] 124 | ---- 125 | fixKafkaParams(kafkaParams: ju.HashMap[String, Object]): Unit 126 | ---- 127 | 128 | `fixKafkaParams` fixes Kafka parameters to prevent any issues with communicating with Kafka on Spark executors. 129 | 130 | CAUTION: FIXME 131 | -------------------------------------------------------------------------------- /spark-streaming-recurringtimer.adoc: -------------------------------------------------------------------------------- 1 | == RecurringTimer 2 | 3 | [source, scala] 4 | ---- 5 | class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: String) 6 | ---- 7 | 8 | `RecurringTimer` (aka *timer*) is a `private[streaming]` class that uses a single daemon thread prefixed `RecurringTimer - [name]` that, once <>, executes `callback` in a loop every `period` time (until it is <>). 9 | 10 | The wait time is achieved by `Clock.waitTillTime` (that makes testing easier). 11 | 12 | [TIP] 13 | ==== 14 | Enable `INFO` or `DEBUG` logging level for `org.apache.spark.streaming.util.RecurringTimer` logger to see what happens inside. 15 | 16 | Add the following line to `conf/log4j.properties`: 17 | 18 | ``` 19 | log4j.logger.org.apache.spark.streaming.util.RecurringTimer=DEBUG 20 | ``` 21 | 22 | Refer to link:../spark-logging.adoc[Logging]. 23 | ==== 24 | 25 | When `RecurringTimer` triggers an action for a `period`, you should see the following DEBUG message in the logs: 26 | 27 | ``` 28 | DEBUG RecurringTimer: Callback for [name] called at time [prevTime] 29 | ``` 30 | 31 | === [[startTime]][[restartTime]] Start and Restart Times 32 | 33 | [source, scala] 34 | ---- 35 | getStartTime(): Long 36 | getRestartTime(originalStartTime: Long): Long 37 | ---- 38 | 39 | `getStartTime` and `getRestartTime` are helper methods that calculate time. 40 | 41 | `getStartTime` calculates a time that is a multiple of the timer's `period` and is right after the current system time. 42 | 43 | NOTE: `getStartTime` is used when link:spark-streaming-jobgenerator.adoc#startFirstTime[JobGenerator is started]. 44 | 45 | `getRestartTime` is similar to `getStartTime` but includes `originalStartTime` input parameter, i.e. it calculates a time as `getStartTime` but shifts the result to accommodate the time gap since `originalStartTime`. 46 | 47 | NOTE: `getRestartTime` is used when link:spark-streaming-jobgenerator.adoc#restarting[JobGenerator is restarted]. 48 | 49 | === [[start]] Starting Timer 50 | 51 | [source, scala] 52 | ---- 53 | start(startTime: Long): Long 54 | start(): Long // <1> 55 | ---- 56 | <1> Uses the internal <> method to calculate `startTime` and calls `start(startTime: Long)`. 57 | 58 | You can start a `RecurringTimer` using `start` methods. 59 | 60 | NOTE: `start()` method uses the internal <> method to calculate `startTime` and calls `start(startTime: Long)`. 61 | 62 | When `start` is called, it sets the internal `nextTime` to the given input parameter `startTime` and starts the internal daemon thread. This is the moment when the clock starts ticking... 63 | 64 | You should see the following INFO message in the logs: 65 | 66 | ``` 67 | INFO RecurringTimer: Started timer for [name] at time [nextTime] 68 | ``` 69 | 70 | === [[stop]] Stopping Timer 71 | 72 | [source, scala] 73 | ---- 74 | stop(interruptTimer: Boolean): Long 75 | ---- 76 | 77 | A timer is stopped using `stop` method. 78 | 79 | NOTE: It is called when link:spark-streaming-jobgenerator.adoc#stop[JobGenerator stops]. 80 | 81 | When called, you should see the following INFO message in the logs: 82 | 83 | ``` 84 | INFO RecurringTimer: Stopped timer for [name] after time [prevTime] 85 | ``` 86 | 87 | `stop` method uses the internal `stopped` flag to mark the stopped state and returns the last `period` for which it was successfully executed (tracked as `prevTime` internally). 88 | 89 | NOTE: Before it fully terminates, it triggers `callback` one more/last time, i.e. `callback` is executed for a `period` after `RecurringTimer` has been (marked) stopped. 90 | 91 | === Fun Fact 92 | 93 | You can execute `org.apache.spark.streaming.util.RecurringTimer` as a command-line standalone application. 94 | 95 | ``` 96 | $ ./bin/spark-class org.apache.spark.streaming.util.RecurringTimer 97 | Setting default log level to "WARN". 98 | To adjust logging level use sc.setLogLevel(newLevel). 99 | INFO RecurringTimer: Started timer for Test at time 1453787444000 100 | INFO RecurringTimer: 1453787444000: 1453787444000 101 | DEBUG RecurringTimer: Callback for Test called at time 1453787444000 102 | INFO RecurringTimer: 1453787445005: 1005 103 | DEBUG RecurringTimer: Callback for Test called at time 1453787445000 104 | INFO RecurringTimer: 1453787446004: 999 105 | DEBUG RecurringTimer: Callback for Test called at time 1453787446000 106 | INFO RecurringTimer: 1453787447005: 1001 107 | DEBUG RecurringTimer: Callback for Test called at time 1453787447000 108 | INFO RecurringTimer: 1453787448000: 995 109 | DEBUG RecurringTimer: Callback for Test called at time 1453787448000 110 | ^C 111 | INFO ShutdownHookManager: Shutdown hook called 112 | INFO ShutdownHookManager: Deleting directory /private/var/folders/0w/kb0d3rqn4zb9fcc91pxhgn8w0000gn/T/spark-71dbd43d-2db3-4527-adb8-f1174d799b0d/repl-a6b9bf12-fec2-4004-9236-3b0ab772cc94 113 | INFO ShutdownHookManager: Deleting directory /private/var/folders/0w/kb0d3rqn4zb9fcc91pxhgn8w0000gn/T/spark-71dbd43d-2db3-4527-adb8-f1174d799b0d 114 | ``` 115 | -------------------------------------------------------------------------------- /spark-streaming-InputInfoTracker.adoc: -------------------------------------------------------------------------------- 1 | == [[InputInfoTracker]] InputInfoTracker 2 | 3 | `InputInfoTracker` tracks batch times and <> for all registered input dstreams. It is used when link:spark-streaming-jobgenerator.adoc#generateJobs[`JobGenerator` submits streaming jobs for a batch interval] and in turn propagated to link:spark-streaming-streaminglisteners.adoc[streaming listeners] (as link:spark-streaming-streaminglisteners.adoc#StreamingListenerEvent[StreamingListenerBatchSubmitted] events). 4 | 5 | NOTE: `InputInfoTracker` is managed by link:spark-streaming-jobscheduler.adoc[JobScheduler], i.e. it is created when link:spark-streaming-jobscheduler.adoc#starting[JobScheduler starts] and link:spark-streaming-jobscheduler.adoc#stopping[is stopped alongside]. 6 | 7 | `InputInfoTracker` uses internal registry <> to maintain the mapping of batch times and link:spark-streaming-inputdstreams.adoc[input dstreams] (i.e. another mapping between input stream ids and <>). 8 | 9 | `InputInfoTracker` accumulates batch statistics for every batch when link:spark-streaming-dstreams.adoc#contract[input streams are computing RDDs] (and call <>). 10 | 11 | [NOTE] 12 | ==== 13 | It is up to input dstreams to have these batch statistics collected (and requires calling <> method explicitly). 14 | 15 | The following input streams report information: 16 | 17 | * link:spark-streaming-kafka-DirectKafkaInputDStream.adoc#compute[DirectKafkaInputDStream] 18 | * link:spark-streaming-receiverinputdstreams.adoc[ReceiverInputDStreams -- Input Streams with Receivers] 19 | * FileInputDStream 20 | ==== 21 | 22 | [TIP] 23 | ==== 24 | Enable `INFO` logging level for `org.apache.spark.streaming.scheduler.InputInfoTracker` logger to see what happens inside. 25 | 26 | Add the following line to `conf/log4j.properties`: 27 | 28 | ``` 29 | log4j.logger.org.apache.spark.streaming.scheduler.InputInfoTracker=INFO 30 | ``` 31 | 32 | Refer to link:../spark-logging.adoc[Logging]. 33 | ==== 34 | 35 | === [[batchTimeToInputInfos]] Batch Intervals and Input DStream Statistics -- `batchTimeToInputInfos` Registry 36 | 37 | [source, scala] 38 | ---- 39 | batchTimeToInputInfos: HashMap[Time, HashMap[Int, StreamInputInfo]] 40 | ---- 41 | 42 | `batchTimeToInputInfos` keeps track of batches (`Time`) with input dstreams (`Int`) that reported their <> per batch. 43 | 44 | === [[reportInfo]] Reporting Input DStream Statistics for Batch -- `reportInfo` Method 45 | 46 | [source, scala] 47 | ---- 48 | reportInfo(batchTime: Time, inputInfo: StreamInputInfo): Unit 49 | ---- 50 | 51 | `reportInfo` adds the input `inputInfo` for the `batchTime` to <>. 52 | 53 | Internally, `reportInfo` accesses the input dstream reports for `batchTime` using the internal `batchTimeToInputInfos` registry (creating a new empty one if `batchTime` has not been registered yet). 54 | 55 | `reportInfo` then makes sure that the `inputInfo` input dstream has not been registered already for the input `batchTime` and throws a `IllegalStateException` otherwise. 56 | 57 | ``` 58 | Input stream [inputStreamId] for batch [batchTime] is already added into InputInfoTracker, this is an illegal state 59 | ``` 60 | 61 | Ultimatelly, `reportInfo` adds the input report to `batchTimeToInputInfos`. 62 | 63 | === [[getInfo]] Requesting Statistics For Input DStreams For Batch -- `getInfo` Method 64 | 65 | [source, scala] 66 | ---- 67 | getInfo(batchTime: Time): Map[Int, StreamInputInfo] 68 | ---- 69 | 70 | `getInfo` returns all the <> for `batchTime`. It returns an empty collection if there are no reports for a batch. 71 | 72 | NOTE: `getInfo` is used when link:spark-streaming-jobgenerator.adoc#generateJobs[`JobGenerator` has successfully generated streaming jobs (and submits the jobs to `JobScheduler`)]. 73 | 74 | === [[cleanup]] Removing Batch Statistics -- `cleanup` Method 75 | 76 | [source, scala] 77 | ---- 78 | cleanup(batchThreshTime: Time): Unit 79 | ---- 80 | 81 | `cleanup` removes statistics for batches older than `batchThreshTime`. It removes the batches from <> registry. 82 | 83 | When executed, you should see the following INFO message (akin to _garbage collection_): 84 | 85 | ``` 86 | INFO InputInfoTracker: remove old batch metadata: [timesToCleanup] 87 | ``` 88 | 89 | === [[StreamInputInfo]] `StreamInputInfo` -- Input Record Statistics 90 | 91 | `StreamInputInfo` is used by link:spark-streaming-inputdstreams.adoc[input dstreams] to <>. 92 | 93 | `StreamInputInfo` contains: 94 | 95 | 1. The id of the input dstream 96 | 2. The number of records in a batch 97 | 3. A metadata (with `Description`) 98 | 99 | NOTE: `Description` is used in `BatchPage` (Details of batch) in web UI for Streaming under `Input Metadata`. 100 | 101 | .Details of batch in web UI for Kafka 0.10 direct stream with Metadata 102 | image::images/spark-streaming-kafka-0-10-webui-details-batch.png[align="center"] 103 | -------------------------------------------------------------------------------- /spark-streaming-webui.adoc: -------------------------------------------------------------------------------- 1 | == web UI and Streaming Statistics Page 2 | 3 | When you link:spark-streaming-streamingcontext.adoc#start[start a Spark Streaming application], you can use link:../spark-webui.adoc[web UI] to monitor streaming statistics in *Streaming* tab (aka _page_). 4 | 5 | .Streaming Tab in web UI 6 | image::images/spark-streaming-webui-streaming-tab.png[align="center"] 7 | 8 | NOTE: The number of completed batches to retain to compute statistics upon is controlled by link:spark-streaming-settings.adoc[spark.streaming.ui.retainedBatches] (and defaults to `1000`). 9 | 10 | The page is made up of three sections (aka _tables_) - the unnamed, top-level one with <> about the streaming application (right below the title *Streaming Statistics*), <> and <>. 11 | 12 | NOTE: The Streaming page uses link:spark-streaming-streaminglisteners.adoc#StreamingJobProgressListener[StreamingJobProgressListener] for most of the information displayed. 13 | 14 | === [[basic-info]] Basic Information 15 | 16 | *Basic Information* section is the top-level section in the Streaming page that offers basic information about the streaming application. 17 | 18 | .Basic Information section in Streaming Page (with Receivers) 19 | image::images/spark-streaming-webui-streaming-statistics.png[align="center"] 20 | 21 | The section shows the link:spark-streaming-dstreamgraph.adoc#batchDuration[batch duration] (in _Running batches of [batch duration]_), and the time it runs for and since link:spark-streaming-streamingcontext.adoc#creating-instance[StreamingContext was created] (_not_ when this streaming application has been started!). 22 | 23 | It shows the number of all *completed batches* (for the entire period since the StreamingContext was started) and *received records* (in parenthesis). These information are later displayed in detail in <> and <> sections. 24 | 25 | Below is the table for link:spark-streaming-streaminglisteners.adoc#retainedBatches[retained batches] (i.e. waiting, running, and completed batches). 26 | 27 | In *Input Rate* row, you can show and hide details of each input stream. 28 | 29 | If there are link:spark-streaming-receiverinputdstreams.adoc[input streams with receivers], the numbers of all the receivers and active ones are displayed (as depicted in the Figure 2 above). 30 | 31 | The average event rate for all registered streams is displayed (as _Avg: [avg] events/sec_). 32 | 33 | ==== [[scheduling-delay]] Scheduling Delay 34 | 35 | *Scheduling Delay* is the time spent from link:spark-streaming-jobscheduler.adoc#submitJobSet[when the collection of streaming jobs for a batch was submitted] to link:spark-streaming-jobscheduler.adoc#JobStarted[when the first streaming job (out of possibly many streaming jobs in the collection) was started]. 36 | 37 | .Scheduling Delay in Streaming Page 38 | image::images/spark-streaming-webui-streaming-page-scheduling-delay.png[align="center"] 39 | 40 | It should be as low as possible meaning that the streaming jobs in batches are scheduled almost instantly. 41 | 42 | NOTE: The values in the timeline (the first column) depict the time between the events link:spark-streaming-streaminglisteners.adoc#StreamingListenerEvent[StreamingListenerBatchSubmitted] and link:spark-streaming-streaminglisteners.adoc#StreamingListenerEvent[StreamingListenerBatchStarted] (with minor yet additional delays to deliver the events). 43 | 44 | You may see increase in scheduling delay in the timeline when streaming jobs are queued up as in the following example: 45 | 46 | [source, scala] 47 | ---- 48 | // batch duration = 5 seconds 49 | val messages: InputDStream[(String, String)] = ... 50 | messages.foreachRDD { rdd => 51 | println(">>> Taking a 15-second sleep") 52 | rdd.foreach(println) 53 | java.util.concurrent.TimeUnit.SECONDS.sleep(15) 54 | } 55 | ---- 56 | 57 | .Scheduling Delay Increased in Streaming Page 58 | image::images/spark-streaming-webui-scheduling-delay-increase.png[align="center"] 59 | 60 | ==== [[processing-time]] Processing Time 61 | 62 | *Processing Time* is the time spent to complete all the streaming jobs of a batch. 63 | 64 | .Batch Processing Time and Batch Intervals 65 | image::images/spark-streaming-batch-processing-time.png[align="center"] 66 | 67 | ==== [[total-delay]] Total Delay 68 | 69 | *Total Delay* is the time spent from submitting to complete all jobs of a batch. 70 | 71 | === [[active-batches]] Active Batches 72 | 73 | *Active Batches* section presents `waitingBatches` and `runningBatches` together. 74 | 75 | === [[completed-batches]] Completed Batches 76 | 77 | *Completed Batches* section presents retained completed batches (using `completedBatchUIData`). 78 | 79 | NOTE: The number of retained batches is controlled by link:spark-streaming-settings.adoc[spark.streaming.ui.retainedBatches]. 80 | 81 | .Completed Batches (limited to 5 elements only) 82 | image::images/spark-streaming-webui-completed-batches.png[align="center"] 83 | 84 | === Example - Kafka Direct Stream in web UI 85 | 86 | .Two Batches with Incoming Data inside for Kafka Direct Stream in web UI (Streaming tab) 87 | image::images/spark-streaming-webui-streaming-tab-kafka-directstream-two-batches.png[align="center"] 88 | 89 | .Two Jobs for Kafka Direct Stream in web UI (Jobs tab) 90 | image::images/spark-streaming-webui-kafka-directinputstream-two-jobs.png[align="center"] 91 | -------------------------------------------------------------------------------- /spark-streaming-kafka.adoc: -------------------------------------------------------------------------------- 1 | == Ingesting Data from Apache Kafka 2 | 3 | Spark Streaming comes with two built-in models of ingesting data from http://kafka.apache.org/[Apache Kafka]: 4 | 5 | * <> 6 | * Using receivers 7 | 8 | There is yet another "middle-ground" approach (so-called unofficial since it is not available by default in Spark Streaming): 9 | 10 | * https://github.com/dibbhatt/kafka-spark-consumer[Kafka Spark Consumer] -- a high-performance Kafka Consumer for Spark Streaming with support for Apache Kafka 0.10. 11 | 12 | === [[no-receivers]] Data Ingestion with no Receivers 13 | 14 | *No-receivers approach* supports the two following modes: 15 | 16 | * <> (using link:spark-streaming-kafka-KafkaUtils.adoc#createDirectStream[KafkaUtils.createDirectStream]) that uses a link:spark-streaming-inputdstreams.adoc[input dstream] that polls for records from Kafka brokers on the driver every batch interval and passes the available topic offsets on to executors for processing. 17 | * *Non-streaming mode* (using link:spark-streaming-kafka-KafkaUtils.adoc#createRDD[KafkaUtils.createRDD]) which simply creates a link:spark-streaming-kafka-KafkaRDD.adoc[KafkaRDD] of key-value pairs, i.e. `RDD[(K, V)]` from the records in topics in Kafka. 18 | 19 | ==== [[streaming-mode]] Streaming mode 20 | 21 | You create link:spark-streaming-kafka-DirectKafkaInputDStream.adoc[DirectKafkaInputDStream] using link:spark-streaming-kafka-KafkaUtils.adoc#createDirectStream[KafkaUtils.createDirectStream]. 22 | 23 | NOTE: Define the types of keys and values in `KafkaUtils.createDirectStream`, e.g. `KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]`, so proper decoders are used to decode messages from Kafka. 24 | 25 | You have to specify `metadata.broker.list` or `bootstrap.servers` (in that order of precedence) for your Kafka environment. `metadata.broker.list` is a comma-separated list of Kafka's (seed) brokers in the format of `:`. 26 | 27 | NOTE: You can start `DirectKafkaInputDStream` regardless of the status of Kafka brokers as it waits until at least one Kafka broker is available. 28 | 29 | [source, scala] 30 | ---- 31 | val conf = new SparkConf().setMaster("local[*]").setAppName("Ingesting Data from Kafka") 32 | conf.set("spark.streaming.ui.retainedBatches", "5") 33 | 34 | // Enable Back Pressure 35 | conf.set("spark.streaming.backpressure.enabled", "true") 36 | 37 | val ssc = new StreamingContext(conf, batchDuration = Seconds(5)) 38 | 39 | // Enable checkpointing 40 | ssc.checkpoint("_checkpoint") 41 | 42 | // You may or may not want to enable some additional DEBUG logging 43 | import org.apache.log4j._ 44 | Logger.getLogger("org.apache.spark.streaming.dstream.DStream").setLevel(Level.DEBUG) 45 | Logger.getLogger("org.apache.spark.streaming.dstream.WindowedDStream").setLevel(Level.DEBUG) 46 | Logger.getLogger("org.apache.spark.streaming.DStreamGraph").setLevel(Level.DEBUG) 47 | Logger.getLogger("org.apache.spark.streaming.scheduler.JobGenerator").setLevel(Level.DEBUG) 48 | 49 | // Connect to Kafka 50 | import org.apache.spark.streaming.kafka.KafkaUtils 51 | import _root_.kafka.serializer.StringDecoder 52 | val kafkaParams = Map("metadata.broker.list" -> "localhost:9092") 53 | val kafkaTopics = Set("spark-topic") 54 | val messages = KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder](ssc, kafkaParams, kafkaTopics) 55 | 56 | // print 10 last messages 57 | messages.print() 58 | 59 | // start streaming computation 60 | ssc.start 61 | ---- 62 | 63 | If `zookeeper.connect` or `group.id` parameters are not set, they are added with their values being empty strings. 64 | 65 | In this mode, you will only see jobs submitted (in the *Jobs* tab in link:spark-webui.adoc[web UI]) when a message comes in. 66 | 67 | .Complete Jobs in web UI for batch time 22:17:15 68 | image::images/spark-streaming-kafka-webui-jobs.png[align="center"] 69 | 70 | It corresponds to *Input size* larger than `0` in the *Streaming* tab in the web UI. 71 | 72 | .Completed Batch in web UI for batch time 22:17:15 73 | image::images/spark-streaming-kafka-webui-streaming.png[align="center"] 74 | 75 | Click the link in Completed Jobs for a batch and you see the details. 76 | 77 | .Details of batch in web UI for batch time 22:17:15 78 | image::images/spark-streaming-kafka-webui-details-batch.png[align="center"] 79 | 80 | === [[spark-streaming-kafka-0-10]] `spark-streaming-kafka-0-10` Library Dependency 81 | 82 | The new API for both Kafka RDD and DStream is in the `spark-streaming-kafka` artifact. Add the following dependency to sbt project to use the streaming integration: 83 | 84 | ``` 85 | libraryDependencies += "org.apache.spark" %% "spark-streaming-kafka-0-10" % "2.0.1" 86 | ``` 87 | 88 | [TIP] 89 | ==== 90 | `spark-streaming-kafka-0-10` module is not included in the CLASSPATH of link:../spark-shell.adoc[spark-shell] so you have to start it with link:../spark-submit.adoc#packages[`--packages` command-line option]. 91 | 92 | ``` 93 | ./bin/spark-shell --packages org.apache.spark:spark-streaming-kafka-0-10_2.11:2.1.0-SNAPSHOT 94 | ``` 95 | ==== 96 | 97 | NOTE: Replace `2.0.1` or `2.1.0-SNAPSHOT` with available version as found at http://search.maven.org/#search%7Cga%7C1%7Ca%3A%22spark-streaming-kafka-0-10_2.11%22[The Central Repository's search]. 98 | 99 | === [[LeaderOffset]] LeaderOffset 100 | 101 | `LeaderOffset` is an internal class to represent an offset on the topic partition on the broker that works on a host and a port. 102 | 103 | === Recommended Reading 104 | 105 | * http://blog.cloudera.com/blog/2015/03/exactly-once-spark-streaming-from-apache-kafka/[Exactly-once Spark Streaming from Apache Kafka] 106 | -------------------------------------------------------------------------------- /spark-streaming-kafka-KafkaRDD.adoc: -------------------------------------------------------------------------------- 1 | == [[KafkaRDD]] KafkaRDD 2 | 3 | `KafkaRDD` is a link:../spark-rdd.adoc[RDD] of Kafka's `ConsumerRecords` from topics in Apache Kafka with support for link:spark-streaming-kafka-HasOffsetRanges.adoc[HasOffsetRanges]. 4 | 5 | NOTE: Kafka's https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/ConsumerRecord.html[ConsumerRecord] holds a topic name, a partition number, the offset of the record in the Kafka partition and the record itself (as a key-value pair). 6 | 7 | `KafkaRDD` uses <> as the partitions that <> (as the host of the topic). 8 | 9 | NOTE: The feature of defining placement preference (aka _location preference_) very well maps a `KafkaRDD` partition to a Kafka topic partition on a Kafka-closest host. 10 | 11 | `KafkaRDD` is created: 12 | 13 | * On demand using link:spark-streaming-kafka-KafkaUtils.adoc#createRDD[KafkaUtils.createRDD] 14 | 15 | * In batches using link:spark-streaming-kafka-KafkaUtils.adoc#createDirectStream[KafkaUtils.createDirectStream] 16 | 17 | `KafkaRDD` is also created when a `DirectKafkaInputDStream` restores `KafkaRDDs` from checkpoint. 18 | 19 | NOTE: `KafkaRDD` is a `private[spark]` class. 20 | 21 | [TIP] 22 | ==== 23 | Enable `INFO` logging level for `org.apache.spark.streaming.kafka010.KafkaRDD` logger to see what happens inside. 24 | 25 | Add the following line to `conf/log4j.properties`: 26 | 27 | ``` 28 | log4j.logger.org.apache.spark.streaming.kafka010.KafkaRDD=INFO 29 | ``` 30 | 31 | Refer to link:../spark-logging.adoc[Logging]. 32 | ==== 33 | 34 | === [[getPartitions]] `getPartitions` Method 35 | 36 | CAUTION: FIXME 37 | 38 | === [[creating-instance]] Creating KafkaRDD Instance 39 | 40 | `KafkaRDD` takes the following when created: 41 | 42 | * [[sc]] link:../spark-sparkcontext.adoc[SparkContext] 43 | * [[kafkaParams]] Collection of Kafka parameters with their values 44 | * [[offsetRanges]] Collection of link:spark-streaming-kafka-HasOffsetRanges.adoc#OffsetRange[OffsetRanges] 45 | * [[preferredHosts]] Kafka's `TopicPartitions` and their hosts 46 | * [[useConsumerCache]] Flag to control whether to use consumer cache 47 | 48 | CAUTION: FIXME Are the hosts in `preferredHosts` Kafka brokers? 49 | 50 | `KafkaRDD` initializes the <>. 51 | 52 | === [[compute]] Computing KafkaRDDPartition (in TaskContext) -- `compute` Method 53 | 54 | [source, scala] 55 | ---- 56 | compute(thePart: Partition, context: TaskContext): Iterator[ConsumerRecord[K, V]] 57 | ---- 58 | 59 | NOTE: `compute` is a part of the link:../spark-rdd.adoc#compute[RDD Contract]. 60 | 61 | `compute` assumes that it works with `thePart` as <> only. It asserts that the offsets are correct, i.e. `fromOffset` is at most `untilOffset`. 62 | 63 | If the beginning and ending offsets are the same, you should see the following INFO message in the logs and `compute` returns an empty collection. 64 | 65 | ``` 66 | INFO KafkaRDD: Beginning offset [fromOffset] is the same as ending offset skipping [topic] [partition] 67 | ``` 68 | 69 | Otherwise, when the beginning and ending offsets are different, a <> is created (for the partition and the input link:../spark-taskscheduler-taskcontext.adoc[TaskContext]) and returned. 70 | 71 | === [[getPreferredLocations]] Getting Placement Preferences of Partition -- `getPreferredLocations` Method 72 | 73 | [source, scala] 74 | ---- 75 | getPreferredLocations(thePart: Partition): Seq[String] 76 | ---- 77 | 78 | NOTE: `getPreferredLocations` is a part of link:../spark-rdd.adoc#getPreferredLocations[RDD contract] to define the placement preferences (aka _preferred locations_) of a partition. 79 | 80 | `getPreferredLocations` casts `thePart` to <>. 81 | 82 | `getPreferredLocations` <>. 83 | 84 | CAUTION: FIXME Use proper name for executors. 85 | 86 | `getPreferredLocations` <> and finds the <> for the partition. 87 | 88 | NOTE: `getPreferredLocations` uses <> that was given when <>. 89 | 90 | If `getPreferredLocations` did not find the preferred host for the partition, all executors are used. Otherwise, `getPreferredLocations` includes only executors on the preferred host. 91 | 92 | If `getPreferredLocations` found no executors, all the executors are considered. 93 | 94 | `getPreferredLocations` returns one matching executor (for the `TopicPartition`) or an empty collection. 95 | 96 | === [[executors]] Creating ExecutorCacheTaskLocations for All Executors in Cluster -- `executors` Internal Method 97 | 98 | [source, scala] 99 | ---- 100 | executors(): Array[ExecutorCacheTaskLocation] 101 | ---- 102 | 103 | `executors` link:spark-BlockManagerMaster.adoc#getPeers[requests `BlockManagerMaster` for all `BlockManager` nodes (peers) in a cluster] (that represent all the executors available). 104 | 105 | NOTE: `executors` uses ``KafkaRDD``'s link:spark-sparkcontext.adoc[SparkContext] to link:spark-blockmanager.adoc#blockManager[access the current `BlockManager`] and in turn link:spark-blockmanager.adoc#master[BlockManagerMaster]. 106 | 107 | `executors` creates `ExecutorCacheTaskLocations` using the peers' hosts and executor ids. 108 | 109 | NOTE: `executors` are sorted by their host names and executor ids. 110 | 111 | CAUTION: FIXME Image for sorted ExecutorCacheTaskLocations. 112 | 113 | NOTE: `executors` is used exclusively when <> (aka _preferred locations_). 114 | 115 | === [[KafkaRDDPartition]] `KafkaRDDPartition` 116 | 117 | `KafkaRDDPartition` is...FIXME 118 | 119 | ==== [[topicPartition]] `topicPartition` 120 | 121 | CAUTION: FIXME 122 | -------------------------------------------------------------------------------- /spark-streaming-statedstreams.adoc: -------------------------------------------------------------------------------- 1 | == StateDStream 2 | 3 | `StateDStream` is the specialized link:spark-streaming-dstreams.adoc[DStream] that is the result of link:spark-streaming-operators-stateful.adoc#updateStateByKey[updateStateByKey] stateful operator. It is a wrapper around a `parent` key-value pair dstream to build stateful pipeline (by means of `updateStateByKey` operator) and as a stateful dstream enables link:spark-streaming-checkpointing.adoc[checkpointing] (and hence requires some additional setup). 4 | 5 | It uses a `parent` key-value pair dstream, <> update state function, a `partitioner`, a flag whether or not to `preservePartitioning` and an optional key-value pair `initialRDD`. 6 | 7 | It works with link:../spark-rdd-StorageLevel.adoc[`MEMORY_ONLY_SER` storage level] enabled. 8 | 9 | The only link:spark-streaming-dstreams.adoc#contract[dependency] of `StateDStream` is the input `parent` key-value pair dstream. 10 | 11 | The link:spark-streaming-dstreams.adoc#contract[slide duration] is exactly the same as that in `parent`. 12 | 13 | It forces link:spark-streaming-checkpointing.adoc[checkpointing] regardless of the current dstream configuration, i.e. the internal link:spark-streaming-dstreams.adoc#checkpointing[mustCheckpoint] is enabled. 14 | 15 | When requested to link:spark-streaming-dstreams.adoc#contract[compute a RDD] it first attempts to get the *state RDD* for the previous batch (using link:spark-streaming-dstreams.adoc#getOrCompute[DStream.getOrCompute]). If there is one, `parent` stream is requested for a RDD for the current batch (using link:spark-streaming-dstreams.adoc#getOrCompute[DStream.getOrCompute]). If `parent` has computed one, <> is called. 16 | 17 | CAUTION: FIXME When could `getOrCompute` *not* return an RDD? How does this apply to the StateDStream? What about the parent's `getOrCompute`? 18 | 19 | If however `parent` has not generated a RDD for the current batch but the state RDD existed, `updateFn` is called for every key of the state RDD to generate a new state per partition (using link:spark-rdd-transformations.adoc#mapPartitions[RDD.mapPartitions]) 20 | 21 | NOTE: No input data for already-running input stream triggers (re)computation of the state RDD (per partition). 22 | 23 | .Computing stateful RDDs (StateDStream.compute) 24 | image::images/spark-streaming-StateDStream-compute.png[align="center"] 25 | 26 | If the state RDD has been found, which means that this is the first input data batch, `parent` stream is requested to link:spark-streaming-dstreams.adoc#getOrCompute[getOrCompute] the RDD for the current batch. 27 | 28 | Otherwise, when no state RDD exists, `parent` stream is requested for a RDD for the current batch (using link:spark-streaming-dstreams.adoc#getOrCompute[DStream.getOrCompute]) and when no RDD was generated for the batch, no computation is triggered. 29 | 30 | NOTE: When the stream processing starts, i.e. no state RDD exists, and there is no input data received, no computation is triggered. 31 | 32 | Given no state RDD and with `parent` RDD computed, when `initialRDD` is `NONE`, the input data batch (as `parent` RDD) is grouped by key (using link:spark-rdd-PairRDDFunctions.adoc#groupByKey[groupByKey] with `partitioner`) and then the update state function `updateFunc` is applied to the partitioned input data (using link:spark-rdd-transformations.adoc#mapPartitions[RDD.mapPartitions]) with `None` state. Otherwise, <> is called. 33 | 34 | === [[updateFunc]] updateFunc - State Update Function 35 | 36 | The signature of `updateFunc` is as follows: 37 | 38 | [source, scala] 39 | ---- 40 | updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)] 41 | ---- 42 | 43 | It should be read as given a collection of triples of a key, new records for the key, and the current state for the key, generate a collection of keys and their state. 44 | 45 | === [[computeUsingPreviousRDD]] computeUsingPreviousRDD 46 | 47 | [source, scala] 48 | ---- 49 | computeUsingPreviousRDD(parentRDD: RDD[(K, V)], prevStateRDD: RDD[(K, S)]): Option[RDD[(K, S)]] 50 | ---- 51 | 52 | The `computeUsingPreviousRDD` method uses `cogroup` and `mapPartitions` to build the final state RDD. 53 | 54 | NOTE: Regardless of the return type `Option[RDD[(K, S)]]` that really allows no state, it will always return _some_ state. 55 | 56 | It first performs `cogroup` of `parentRDD` and `prevStateRDD` using the constructor's `partitioner` so it has a pair of iterators of elements of each RDDs per _every_ key. 57 | 58 | NOTE: It is acceptable to end up with keys that have no new records per batch, but these keys do have a state (since they were received previously when no state might have been built yet). 59 | 60 | [NOTE] 61 | ==== 62 | The signature of `cogroup` is as follows and applies to key-value pair RDDs, i.e. `RDD[(K, V)]`. 63 | 64 | [source, scala] 65 | ---- 66 | cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Iterable[V], Iterable[W]))] 67 | ---- 68 | ==== 69 | 70 | It defines an internal update function `finalFunc` that maps over the collection of all the keys, new records per key, and at-most-one-element state per key to build new iterator that ensures that: 71 | 72 | 1. a state per key exists (it is `None` or the state built so far) 73 | 2. the _lazy_ iterable of new records is transformed into an _eager_ sequence. 74 | 75 | CAUTION: FIXME Why is the transformation from an Iterable into a Seq so important? Why could not the constructor's <> accept the former? 76 | 77 | With every triple per every key, the internal update function calls the constructor's <>. 78 | 79 | The state RDD is a cogrouped RDD (on `parentRDD` and `prevStateRDD` using the constructor's `partitioner`) with every element per partition mapped over using the internal update function `finalFunc` and the constructor's `preservePartitioning` (through `mapPartitions`). 80 | 81 | CAUTION: FIXME Why is `preservePartitioning` important? What happens when `mapPartitions` does not preserve partitioning (which by default it does *not*!) 82 | -------------------------------------------------------------------------------- /spark-streaming-backpressure.adoc: -------------------------------------------------------------------------------- 1 | == Backpressure (Back Pressure) 2 | 3 | Quoting https://youtu.be/ng-jZhVK3ro[TD from his talk about Spark Streaming]: 4 | 5 | > Backpressure is to make applications robust against data surges. 6 | 7 | With backpressure you can guarantee that your Spark Streaming application is *stable*, i.e. receives data only as fast as it can process it. 8 | 9 | NOTE: Backpressure shifts the trouble of buffering input records to the sender so it keeps records until they could be processed by a streaming application. You could alternatively use link:spark-streaming-dynamic-allocation.adoc[dynamic allocation] feature in Spark Streaming to increase the capacity of streaming infrastructure without slowing down the senders. 10 | 11 | Backpressure is disabled by default and can be turned on using link:spark-streaming-settings.adoc#back-pressure[spark.streaming.backpressure.enabled] setting. 12 | 13 | You can monitor a streaming application using link:spark-streaming-webui.adoc[web UI]. It is important to ensure that the link:spark-streaming-webui.adoc#processing-time[batch processing time] is shorter than the link:spark-streaming.adoc#batch-interval[batch interval]. Backpressure introduces a *feedback loop* so the streaming system can adapt to longer processing times and avoid instability. 14 | 15 | NOTE: Backpressure is available since Spark 1.5. 16 | 17 | === [[RateController]] RateController 18 | 19 | TIP: Read up on https://en.wikipedia.org/wiki/Back_pressure[back pressure] in Wikipedia. 20 | 21 | `RateController` is a contract for single-dstream link:spark-streaming-streaminglisteners.adoc[StreamingListeners] that listens to link:spark-streaming-streaminglisteners.adoc#onBatchCompleted[batch completed updates] for a dstream and maintains a *rate limit*, i.e. an estimate of the speed at which this stream should ingest messages. With every batch completed update event it calculates the current processing rate and estimates the correct receving rate. 22 | 23 | NOTE: `RateController` works for a single dstream and requires a <>. 24 | 25 | The contract says that RateControllers offer the following method: 26 | 27 | [source, scala] 28 | ---- 29 | protected def publish(rate: Long): Unit 30 | ---- 31 | 32 | When created, it creates a daemon single-thread executor service called *stream-rate-update* and initializes the internal `rateLimit` counter which is the current message-ingestion speed. 33 | 34 | When a batch completed update happens, a `RateController` grabs `processingEndTime`, `processingDelay`, `schedulingDelay`, and `numRecords` processed for the batch, computes a rate limit and publishes the current value. The computed value is set as the present rate limit, and published (using the sole abstract `publish` method). 35 | 36 | Computing a rate limit happens using the RateEstimator's `compute` method. 37 | 38 | CAUTION: FIXME Where is this used? What are the use cases? 39 | 40 | link:spark-streaming-inputdstreams.adoc[InputDStreams] can define a `RateController` that is registered to link:spark-streaming-jobscheduler.adoc[JobScheduler]'s `listenerBus` (using `ssc.addStreamingListener`) when link:spark-streaming-jobscheduler.adoc[JobScheduler] starts. 41 | 42 | === [[RateEstimator]] RateEstimator 43 | 44 | `RateEstimator` computes the rate given the input `time`, `elements`, `processingDelay`, and `schedulingDelay`. 45 | 46 | It is an abstract class with the following abstract method: 47 | 48 | [source, scala] 49 | ---- 50 | def compute( 51 | time: Long, 52 | elements: Long, 53 | processingDelay: Long, 54 | schedulingDelay: Long): Option[Double] 55 | ---- 56 | 57 | You can control what `RateEstimator` to use through link:spark-streaming-settings.adoc[spark.streaming.backpressure.rateEstimator] setting. 58 | 59 | The only possible `RateEstimator` to use is the <>. 60 | 61 | === [[PIDRateEstimator]] PID Rate Estimator 62 | 63 | *PID Rate Estimator* (represented as `PIDRateEstimator`) implements a https://en.wikipedia.org/wiki/PID_controller[proportional-integral-derivative (PID) controller] which acts on the speed of ingestion of records into an input dstream. 64 | 65 | WARNING: The *PID rate estimator* is the only possible estimator. All other rate estimators lead to `IllegalArgumentException` being thrown. 66 | 67 | It uses the following settings: 68 | 69 | * `spark.streaming.backpressure.pid.proportional` (default: 1.0) can be 0 or greater. 70 | * `spark.streaming.backpressure.pid.integral` (default: 0.2) can be 0 or greater. 71 | * `spark.streaming.backpressure.pid.derived` (default: 0.0) can be 0 or greater. 72 | * `spark.streaming.backpressure.pid.minRate` (default: 100) must be greater than 0. 73 | 74 | NOTE: The PID rate estimator is used by link:spark-streaming-kafka-DirectKafkaInputDStream.adoc#back-pressure[DirectKafkaInputDStream] and link:spark-streaming-receiverinputdstreams.adoc#back-pressure[input dstreams with receivers (aka ReceiverInputDStreams)]. 75 | 76 | [TIP] 77 | ==== 78 | Enable `INFO` or `TRACE` logging level for `org.apache.spark.streaming.scheduler.rate.PIDRateEstimator` logger to see what happens inside. 79 | 80 | Add the following line to `conf/log4j.properties`: 81 | 82 | ``` 83 | log4j.logger.org.apache.spark.streaming.scheduler.rate.PIDRateEstimator=TRACE 84 | ``` 85 | 86 | Refer to link:../spark-logging.adoc[Logging]. 87 | ==== 88 | 89 | When the PID rate estimator is created you should see the following INFO message in the logs: 90 | 91 | ``` 92 | INFO PIDRateEstimator: Created PIDRateEstimator with proportional = [proportional], integral = [integral], derivative = [derivative], min rate = [minRate] 93 | ``` 94 | 95 | When the pid rate estimator computes the rate limit for the current time, you should see the following TRACE message in the logs: 96 | 97 | ``` 98 | TRACE PIDRateEstimator: 99 | time = [time], # records = [numElements], processing time = [processingDelay], scheduling delay = [schedulingDelay] 100 | ``` 101 | 102 | If the time to compute the current rate limit for is before the latest time or the number of records is 0 or less, or processing delay is 0 or less, the rate estimation is skipped. You should see the following TRACE message in the logs: 103 | 104 | ``` 105 | TRACE PIDRateEstimator: Rate estimation skipped 106 | ``` 107 | 108 | And no rate limit is returned. 109 | 110 | Otherwise, when this is to compute the rate estimation for next time and there are records processed as well as the processing delay is positive, it computes the rate estimate. 111 | 112 | Once the new rate has already been computed, you should see the following TRACE message in the logs: 113 | 114 | ``` 115 | TRACE PIDRateEstimator: 116 | latestRate = [latestRate], error = [error] 117 | latestError = [latestError], historicalError = [historicalError] 118 | delaySinceUpdate = [delaySinceUpdate], dError = [dError] 119 | ``` 120 | 121 | If it was the first computation of the limit rate, you should see the following TRACE message in the logs: 122 | 123 | ``` 124 | TRACE PIDRateEstimator: First run, rate estimation skipped 125 | ``` 126 | 127 | No rate limit is returned. 128 | 129 | Otherwise, when it is another limit rate, you should see the following TRACE message in the logs: 130 | 131 | ``` 132 | TRACE PIDRateEstimator: New rate = [newRate] 133 | ``` 134 | 135 | And the current rate limit is returned. 136 | -------------------------------------------------------------------------------- /spark-streaming-kafka-ConsumerStrategy.adoc: -------------------------------------------------------------------------------- 1 | == [[ConsumerStrategy]] `ConsumerStrategy` -- Kafka Consumers' Post-Configuration API 2 | 3 | `ConsumerStrategy` is a <> to create Kafka Consumers in a Spark Streaming application that allows for their custom configuration after the consumers have been created. 4 | 5 | NOTE: Kafka consumers read records from topic partitions in a Kafka cluster. 6 | 7 | `ConsumerStrategy[K, V]` is an abstract class with two methods, i.e. <> and <>. 8 | 9 | .ConsumerStrategy Contract and DirectKafkaInputDStream 10 | [cols="1,2",options="header",width="100%"] 11 | |=== 12 | | Consumer Strategy | DirectKafkaInputDStream Usage 13 | | <> | Used when a link:spark-streaming-kafka-DirectKafkaInputDStream.adoc#creating-instance[`DirectKafkaInputDStream` is created] to initialize internal state. 14 | | <> | Used to link:spark-streaming-kafka-DirectKafkaInputDStream.adoc#consumer[create a Kafka consumer (in `DirectKafkaInputDStream`)] 15 | |=== 16 | 17 | The following table are the Kafka Consumer strategies currently available in Spark 2.0. 18 | 19 | .Kafka Consumer Strategies in Spark Streaming 20 | [cols="1,2",options="header",width="100%"] 21 | |=== 22 | | Consumer Strategy | Description 23 | | <> | 24 | | <> | 25 | | <> | 26 | |=== 27 | 28 | You can access the predefined `ConsumerStrategy` implementations using link:spark-streaming-kafka-ConsumerStrategies.adoc[ConsumerStrategies factory object]. 29 | 30 | [source, scala] 31 | ---- 32 | import org.apache.spark.streaming.kafka010.ConsumerStrategies 33 | 34 | val topics = List("topic1") 35 | import org.apache.kafka.common.serialization.StringDeserializer 36 | val kafkaParams = Map( 37 | "bootstrap.servers" -> "localhost:9092", 38 | "key.deserializer" -> classOf[StringDeserializer], 39 | "value.deserializer" -> classOf[StringDeserializer], 40 | "group.id" -> "spark-streaming-notes", 41 | "auto.offset.reset" -> "earliest" 42 | ) 43 | import org.apache.kafka.common.TopicPartition 44 | val offsets = Map(new TopicPartition("topic3", 0) -> 2L) 45 | 46 | val subscribeStrategy = ConsumerStrategies.Subscribe[String, String](topics, kafkaParams, offsets) 47 | ---- 48 | 49 | === [[contract]] ConsumerStrategy Contract 50 | 51 | ==== [[executorKafkaParams]] `executorKafkaParams` Method 52 | 53 | [source, scala] 54 | ---- 55 | executorKafkaParams: ju.Map[String, Object] 56 | ---- 57 | 58 | ==== [[onStart]] `onStart` Method 59 | 60 | [source, scala] 61 | ---- 62 | onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] 63 | ---- 64 | 65 | === [[Assign]] Assign Strategy 66 | 67 | [source, scala] 68 | ---- 69 | class Assign[K, V]( 70 | topicPartitions: java.util.Collection[TopicPartition], 71 | kafkaParams: java.util.Map[String, Object], 72 | offsets: java.util.Map[TopicPartition, java.util.Long] 73 | ) extends ConsumerStrategy[K, V] 74 | ---- 75 | 76 | `Assign` returns the input `kafkaParams` directly from <> method. 77 | 78 | For `onStart`, `Assign` creates a `KafkaConsumer` (with `kafkaParams`) and explicitly assigns the list of partitions `topicPartitions` to this consumer (using Kafka's link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#assign(java.util.Collection)++[KafkaConsumer.assign] method). It then overrides the fetch offsets that the consumer will use (on the next link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll(long)++[poll]) to ``onStart``'s input `currentOffsets` or `offsets` whatever is not empty (using Kafka's link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#seek(org.apache.kafka.common.TopicPartition,%20long)++[KafkaConsumer.seek] method). 79 | 80 | === [[Subscribe]] Subscribe Strategy 81 | 82 | [source, scala] 83 | ---- 84 | class Subscribe[K, V]( 85 | topics: java.util.Collection[jl.String], 86 | kafkaParams: java.util.Map[String, Object], 87 | offsets: java.util.Map[TopicPartition, java.util.Long] 88 | ) extends ConsumerStrategy[K, V] 89 | ---- 90 | 91 | `Subscribe` returns the input `kafkaParams` directly from <> method. 92 | 93 | For `onStart`, `Subscribe` creates a `KafkaConsumer` (with `kafkaParams`) and subscribes to `topics` (using Kafka's link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#subscribe(java.util.Collection)++[KafkaConsumer.subscribe] method). For non-empty `currentOffsets` or `offsets` (whatever is not empty in that order), `onStart` polls data for topics or partitions (using Kafka's link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll(long)++[KafkaConsumer.poll] method). It then overrides the fetch offsets that the consumer will use (on the next link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll(long)++[poll]) to ``onStart``'s input `currentOffsets` or `offsets` whatever is not empty (using Kafka's link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#seek(org.apache.kafka.common.TopicPartition,%20long)++[KafkaConsumer.seek] method). 94 | 95 | TIP: You can suppress Kafka's `NoOffsetForPartitionException` with Kafka's `auto.offset.reset` setting set to `NONE` in `kafkaParams`. 96 | 97 | In case of Kafka's `NoOffsetForPartitionException` with exception suppression enabled, you can see the following WARN message in the logs: 98 | 99 | ``` 100 | WARN Catching NoOffsetForPartitionException since auto.offset.reset is none. See KAFKA-3370 101 | ``` 102 | 103 | TIP: Read through https://issues.apache.org/jira/browse/KAFKA-3370[KAFKA-3370: Add options to auto.offset.reset to reset offsets upon initialization only] 104 | 105 | [source, scala] 106 | ---- 107 | ??? FIXME Example with the WARN above 108 | ---- 109 | 110 | === [[SubscribePattern]] SubscribePattern Strategy 111 | 112 | [source, scala] 113 | ---- 114 | class SubscribePattern[K, V]( 115 | pattern: java.util.regex.Pattern, 116 | kafkaParams: java.util.Map[String, Object], 117 | offsets: java.util.Map[TopicPartition, java.util.Long] 118 | ) extends ConsumerStrategy[K, V] 119 | ---- 120 | 121 | `SubscribePattern` returns the input `kafkaParams` directly from <> method. 122 | 123 | For `onStart`, `SubscribePattern` creates a `KafkaConsumer` (with `kafkaParams`) and subscribes to `pattern` topics with Kafka's internal `NoOpConsumerRebalanceListener` (using Kafka's link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#subscribe(java.util.Collection,%20org.apache.kafka.clients.consumer.ConsumerRebalanceListener)++[KafkaConsumer.subscribe] method). 124 | 125 | NOTE: The only difference between <> and <> Consumer strategies is the use of Kafka's link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#subscribe(java.util.Collection,%20org.apache.kafka.clients.consumer.ConsumerRebalanceListener)++[KafkaConsumer.subscribe(Collection, ConsumerRebalanceListener)] and link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#subscribe(java.util.Collection)++[KafkaConsumer.subscribe(Collection)] methods, respectively. 126 | -------------------------------------------------------------------------------- /spark-streaming-operators-stateful.adoc: -------------------------------------------------------------------------------- 1 | == Working with State using Stateful Operators 2 | 3 | > Building Stateful Stream Processing Pipelines using Spark (Streaming) 4 | 5 | *Stateful operators* (like <> or <>) are part of the set of additional operators available on link:spark-streaming-dstreams.adoc[DStreams] of key-value pairs, i.e. instances of `DStream[(K, V)]`. They allow you to build *stateful stream processing pipelines* and are also called https://databricks.gitbooks.io/databricks-spark-reference-applications/content/logs_analyzer/chapter1/total.html[cumulative calculations]. 6 | 7 | The motivation for the stateful operators is that by design streaming operators are stateless and know nothing about the previous records and hence a state. If you'd like to react to new records appropriately given the previous records you would have to resort to using persistent storages outside Spark Streaming. 8 | 9 | NOTE: These additional operators are available automatically on pair DStreams through the Scala implicit conversion `DStream.toPairDStreamFunctions`. 10 | 11 | === [[mapWithState]] mapWithState Operator 12 | 13 | [source, scala] 14 | ---- 15 | mapWithState(spec: StateSpec[K, V, ST, MT]): MapWithStateDStream[K, V, ST, MT] 16 | ---- 17 | 18 | You create <> instances for `mapWithState` operator using the factory methods <>. 19 | 20 | `mapWithState` creates a link:spark-streaming-mapwithstatedstreams.adoc[MapWithStateDStream] dstream. 21 | 22 | ==== [[mapWithState-example]] mapWithState Example 23 | 24 | [source, scala] 25 | ---- 26 | import org.apache.spark.streaming.{ StreamingContext, Seconds } 27 | val ssc = new StreamingContext(sc, batchDuration = Seconds(5)) 28 | 29 | // checkpointing is mandatory 30 | ssc.checkpoint("_checkpoints") 31 | 32 | val rdd = sc.parallelize(0 to 9).map(n => (n, n % 2 toString)) 33 | import org.apache.spark.streaming.dstream.ConstantInputDStream 34 | val sessions = new ConstantInputDStream(ssc, rdd) 35 | 36 | import org.apache.spark.streaming.{State, StateSpec, Time} 37 | val updateState = (batchTime: Time, key: Int, value: Option[String], state: State[Int]) => { 38 | println(s">>> batchTime = $batchTime") 39 | println(s">>> key = $key") 40 | println(s">>> value = $value") 41 | println(s">>> state = $state") 42 | val sum = value.getOrElse("").size + state.getOption.getOrElse(0) 43 | state.update(sum) 44 | Some((key, value, sum)) // mapped value 45 | } 46 | val spec = StateSpec.function(updateState) 47 | val mappedStatefulStream = sessions.mapWithState(spec) 48 | 49 | mappedStatefulStream.print() 50 | ---- 51 | 52 | ==== [[StateSpec]] StateSpec - Specification of mapWithState 53 | 54 | `StateSpec` is a state specification of <> and describes how the corresponding state RDD should work (RDD-wise) and maintain a state (streaming-wise). 55 | 56 | NOTE: `StateSpec` is a Scala `sealed abstract class` and hence all the implementations are in the same compilation unit, i.e. source file. 57 | 58 | It requires the following: 59 | 60 | * `initialState` which is the initial state of the transformation, i.e. paired `RDD[(KeyType, StateType)`. 61 | 62 | * `numPartitions` which is the number of partitions of the state RDD. It uses link:../spark-rdd-HashPartitioner.adoc[HashPartitioner] with the given number of partitions. 63 | 64 | * `partitioner` which is the partitioner of the state RDD. 65 | 66 | * `timeout` that sets the idle duration after which the state of an _idle_ key will be removed. A key and its state is considered _idle_ if it has not received any data for at least the given idle duration. 67 | 68 | ===== [[StateSpec-function]] StateSpec.function Factory Methods 69 | 70 | You create `StateSpec` instances using the factory methods `StateSpec.function` (that differ in whether or not you want to access a batch time and return an optional mapped value): 71 | 72 | [source, scala] 73 | ---- 74 | // batch time and optional mapped return value 75 | StateSpec.function(f: (Time, K, Option[V], State[S]) => Option[M]): StateSpec[K, V, S, M] 76 | 77 | // no batch time and mandatory mapped value 78 | StateSpec.function(f: (K, Option[V], State[S]) => M): StateSpec[K, V, S, M] 79 | ---- 80 | 81 | Internally, the `StateSpec.function` executes `ClosureCleaner.clean` to clean up the input function `f` and makes sure that `f` can be serialized and sent over the wire (cf. link:spark-sparkcontext.adoc#closure-cleaning[Closure Cleaning (clean method)]). It will throw an exception when the input function cannot be serialized. 82 | 83 | === [[updateStateByKey]] updateStateByKey Operator 84 | 85 | [source, scala] 86 | ---- 87 | updateStateByKey(updateFn: (Seq[V], Option[S]) => Option[S]): DStream[(K, S)] // <1> 88 | updateStateByKey(updateFn: (Seq[V], Option[S]) => Option[S], 89 | numPartitions: Int): DStream[(K, S)] // <2> 90 | updateStateByKey(updateFn: (Seq[V], Option[S]) => Option[S], 91 | partitioner: Partitioner): DStream[(K, S)] // <3> 92 | updateStateByKey(updateFn: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)], 93 | partitioner: Partitioner, 94 | rememberPartitioner: Boolean): DStream[(K, S)] // <4> 95 | updateStateByKey(updateFn: (Seq[V], Option[S]) => Option[S], 96 | partitioner: Partitioner, 97 | initialRDD: RDD[(K, S)]): DStream[(K, S)] 98 | updateStateByKey(updateFn: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)], 99 | partitioner: Partitioner, 100 | rememberPartitioner: Boolean, 101 | initialRDD: RDD[(K, S)]): DStream[(K, S)] 102 | ---- 103 | <1> When not specified explicitly, the partitioner used is link:../spark-rdd-HashPartitioner.adoc[HashPartitioner] with the number of partitions being the default level of parallelism of a link:spark-TaskScheduler.adoc[Task Scheduler]. 104 | <2> You may however specify the number of partitions explicitly for link:../spark-rdd-HashPartitioner.adoc[HashPartitioner] to use. 105 | <3> This is the "canonical" `updateStateByKey` the other two variants (without a partitioner or the number of partitions) use that allows specifying a partitioner explicitly. It then executes the "last" `updateStateByKey` with `rememberPartitioner` enabled. 106 | <4> The "last" `updateStateByKey` 107 | 108 | `updateStateByKey` stateful operator allows for maintaining per-key state and updating it using `updateFn`. The `updateFn` is called for each key, and uses new data and existing state of the key, to generate an updated state. 109 | 110 | TIP: You should use <> instead as a much performance effective alternative. 111 | 112 | NOTE: Please consult https://issues.apache.org/jira/browse/SPARK-2629[SPARK-2629 Improved state management for Spark Streaming] for performance-related changes to the operator. 113 | 114 | The state update function `updateFn` scans every key and generates a new state for every key given a collection of values per key in a batch and the current state for the key (if exists). 115 | 116 | .updateStateByKey in motion 117 | image::images/spark-streaming-updateStateByKey.png[align="center"] 118 | 119 | Internally, `updateStateByKey` executes link:spark-sparkcontext.adoc#closure-cleaning[SparkContext.clean] on the input function `updateFn`. 120 | 121 | NOTE: The operator does not offer any timeout of idle data. 122 | 123 | `updateStateByKey` creates a link:spark-streaming-statedstreams.adoc[StateDStream] stream. 124 | 125 | ==== [[updateStateByKey-example]] updateStateByKey Example 126 | 127 | [source, scala] 128 | ---- 129 | import org.apache.spark.streaming.{ StreamingContext, Seconds } 130 | val ssc = new StreamingContext(sc, batchDuration = Seconds(5)) 131 | 132 | // checkpointing is mandatory 133 | ssc.checkpoint("_checkpoints") 134 | 135 | val rdd = sc.parallelize(0 to 9).map(n => (n, n % 2 toString)) 136 | import org.apache.spark.streaming.dstream.ConstantInputDStream 137 | val clicks = new ConstantInputDStream(ssc, rdd) 138 | 139 | // helper functions 140 | val inc = (n: Int) => n + 1 141 | def buildState: Option[Int] = { 142 | println(s">>> >>> Initial execution to build state or state is deliberately uninitialized yet") 143 | println(s">>> >>> Building the state being the number of calls to update state function, i.e. the number of batches") 144 | Some(1) 145 | } 146 | 147 | // the state update function 148 | val updateFn: (Seq[String], Option[Int]) => Option[Int] = { case (vs, state) => 149 | println(s">>> update state function with values only, i.e. no keys") 150 | println(s">>> vs = $vs") 151 | println(s">>> state = $state") 152 | state.map(inc).orElse(buildState) 153 | } 154 | val statefulStream = clicks.updateStateByKey(updateFn) 155 | statefulStream.print() 156 | ---- 157 | -------------------------------------------------------------------------------- /spark-streaming-receiversupervisors.adoc: -------------------------------------------------------------------------------- 1 | == ReceiverSupervisors 2 | 3 | `ReceiverSupervisor` is an (abstract) handler object that is responsible for supervising a link:spark-streaming-receivers.adoc[receiver] (that runs on the worker). It assumes that implementations offer concrete methods to push received data to Spark. 4 | 5 | NOTE: link:spark-streaming-receivers.adoc[Receiver]'s `store` methods pass calls to respective `push` methods of ReceiverSupervisors. 6 | 7 | NOTE: link:spark-streaming-receivertracker.adoc[ReceiverTracker] starts a ReceiverSupervisor per receiver. 8 | 9 | `ReceiverSupervisor` can be started and stopped. When a supervisor is started, it calls (empty by default) `onStart()` and `startReceiver()` afterwards. 10 | 11 | It attaches itself to the receiver it is a supervisor of (using `Receiver.attachSupervisor`). That is how a receiver knows about its supervisor (and can hence offer the `store` and management methods). 12 | 13 | === [[contract]] ReceiverSupervisor Contract 14 | 15 | `ReceiverSupervisor` is a `private[streaming] abstract class` that assumes that concrete implementations offer the following *push methods*: 16 | 17 | * `pushBytes` 18 | * `pushIterator` 19 | * `pushArrayBuffer` 20 | 21 | There are the other methods required: 22 | 23 | * `createBlockGenerator` 24 | * `reportError` 25 | * `onReceiverStart` 26 | 27 | === [[starting-receivers]] Starting Receivers 28 | 29 | `startReceiver()` calls (abstract) `onReceiverStart()`. When `true` (it is unknown at this point to know when it is `true` or `false` since it is an abstract method - see <> for the default implementation), it prints the following INFO message to the logs: 30 | 31 | ``` 32 | INFO Starting receiver 33 | ``` 34 | 35 | The receiver's `onStart()` is called and another INFO message appears in the logs: 36 | 37 | ``` 38 | INFO Called receiver onStart 39 | ``` 40 | 41 | If however `onReceiverStart()` returns `false`, the supervisor stops (using `stop`). 42 | 43 | === [[stopping-receivers]] Stopping Receivers 44 | 45 | `stop` method is called with a message and an optional cause of the stop (called `error`). It calls `stopReceiver` method that prints the INFO message and checks the state of the receiver to react appropriately. 46 | 47 | When the receiver is in `Started` state, `stopReceiver` calls `Receiver.onStop()`, prints the following INFO message, and `onReceiverStop(message, error)`. 48 | 49 | ``` 50 | INFO Called receiver onStop 51 | ``` 52 | 53 | === [[restarting-receivers]] Restarting Receivers 54 | 55 | A `ReceiverSupervisor` uses link:spark-streaming-settings.adoc[spark.streaming.receiverRestartDelay] to restart the receiver with delay. 56 | 57 | NOTE: Receivers can request to be restarted using `restart` methods. 58 | 59 | When requested to restart a receiver, it uses a separate thread to perform it asynchronously. It prints the WARNING message to the logs: 60 | 61 | ``` 62 | WARNING Restarting receiver with delay [delay] ms: [message] 63 | ``` 64 | 65 | It then stops the receiver, sleeps for `delay` milliseconds and starts the receiver (using `startReceiver()`). 66 | 67 | You should see the following messages in the logs: 68 | 69 | ``` 70 | DEBUG Sleeping for [delay] 71 | INFO Starting receiver again 72 | INFO Receiver started again 73 | ``` 74 | 75 | CAUTION: FIXME What is a backend data store? 76 | 77 | === [[awaitTermination]] Awaiting Termination 78 | 79 | `awaitTermination` method blocks the current thread to wait for the receiver to be stopped. 80 | 81 | NOTE: ReceiverTracker uses `awaitTermination` to wait for receivers to stop (see link:spark-streaming-receivertracker.adoc#ReceiverTrackerEndpoint-StartAllReceivers[StartAllReceivers]). 82 | 83 | When called, you should see the following INFO message in the logs: 84 | 85 | ``` 86 | INFO Waiting for receiver to be stopped 87 | ``` 88 | 89 | If a receiver has terminated successfully, you should see the following INFO message in the logs: 90 | 91 | ``` 92 | INFO Stopped receiver without error 93 | ``` 94 | 95 | Otherwise, you should see the ERROR message in the logs: 96 | 97 | ``` 98 | ERROR Stopped receiver with error: [stoppingError] 99 | ``` 100 | 101 | `stoppingError` is the exception associated with the stopping of the receiver and is rethrown. 102 | 103 | NOTE: Internally, ReceiverSupervisor uses https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CountDownLatch.html[java.util.concurrent.CountDownLatch] with count `1` to await the termination. 104 | 105 | ==== Internals - How to count stopLatch down 106 | 107 | `stopLatch` is decremented when ReceiverSupervisor's `stop` is called which is in the following cases: 108 | 109 | * When a receiver itself calls `stop(message: String)` or `stop(message: String, error: Throwable)` 110 | * When <> returns `false` or `NonFatal` (less severe) exception is thrown in `ReceiverSupervisor.startReceiver`. 111 | * When link:spark-streaming-receivertracker.adoc#stopping[ReceiverTracker.stop] is called that posts `StopAllReceivers` message to `ReceiverTrackerEndpoint`. It in turn sends `StopReceiver` to the `ReceiverSupervisorImpl` for every `ReceiverSupervisor` that calls `ReceiverSupervisorImpl.stop`. 112 | 113 | [CAUTION] 114 | ==== 115 | FIXME Prepare exercises 116 | 117 | * for a receiver to call `stop(message: String)` when a custom "TERMINATE" message arrives 118 | * send `StopReceiver` to a ReceiverTracker 119 | ==== 120 | 121 | === [[ReceiverSupervisorImpl]] ReceiverSupervisorImpl 122 | 123 | `ReceiverSupervisorImpl` is the implementation of <>. 124 | 125 | NOTE: A dedicated `ReceiverSupervisorImpl` is started for every receiver when <>. See <>. 126 | 127 | It communicates with <> that runs on the driver (by posting messages using the <>). 128 | 129 | [TIP] 130 | ==== 131 | Enable `DEBUG` logging level for `org.apache.spark.streaming.receiver.ReceiverSupervisorImpl` logger to see what happens in `ReceiverSupervisorImpl`. 132 | 133 | Add the following line to `conf/log4j.properties`: 134 | 135 | ``` 136 | log4j.logger.org.apache.spark.streaming.receiver.ReceiverSupervisorImpl=DEBUG 137 | ``` 138 | ==== 139 | 140 | ==== [[ReceiverSupervisorImpl-push-methods]] push Methods 141 | 142 | <>, i.e. `pushArrayBuffer`, `pushIterator`, and `pushBytes` solely pass calls on to <>. 143 | 144 | ==== [[ReceiverSupervisorImpl-onReceiverStart]] ReceiverSupervisorImpl.onReceiverStart 145 | 146 | `ReceiverSupervisorImpl.onReceiverStart` sends a blocking `RegisterReceiver` message to link:spark-streaming-receivertracker.adoc[ReceiverTracker] that responds with a boolean value. 147 | 148 | ==== [[ReceiverSupervisorImpl-currentRateLimit]] Current Rate Limit 149 | 150 | `getCurrentRateLimit` controls the current rate limit. It asks the `BlockGenerator` for the value (using `getCurrentLimit`). 151 | 152 | ==== [[ReceiverSupervisorImpl-receivedBlockHandler]] ReceivedBlockHandler 153 | 154 | `ReceiverSupervisorImpl` uses the internal field `receivedBlockHandler` for link:spark-streaming-receivedblockhandlers.adoc[ReceivedBlockHandler] to use. 155 | 156 | It defaults to link:spark-streaming-receivedblockhandlers.adoc#BlockManagerBasedBlockHandler[BlockManagerBasedBlockHandler], but could use link:spark-streaming-receivedblockhandlers.adoc#WriteAheadLogBasedBlockHandler[WriteAheadLogBasedBlockHandler] instead when link:spark-streaming-settings.adoc[spark.streaming.receiver.writeAheadLog.enable] is `true`. 157 | 158 | It uses `ReceivedBlockHandler` to `storeBlock` (see link:spark-streaming-receivedblockhandlers.adoc#contract[ReceivedBlockHandler Contract] for more coverage and <> in this document). 159 | 160 | ==== [[ReceiverSupervisorImpl-pushAndReportBlock]] ReceiverSupervisorImpl.pushAndReportBlock 161 | 162 | `ReceiverSupervisorImpl.pushAndReportBlock(receivedBlock: ReceivedBlock, metadataOption: Option[Any], blockIdOption: Option[StreamBlockId])` stores `receivedBlock` using `ReceivedBlockHandler.storeBlock` and reports it to the driver. 163 | 164 | NOTE: `ReceiverSupervisorImpl.pushAndReportBlock` is only used by the <>, i.e. `pushArrayBuffer`, `pushIterator`, and `pushBytes`. Calling the method is actually all they do. 165 | 166 | When it calls `ReceivedBlockHandler.storeBlock`, you should see the following DEBUG message in the logs: 167 | 168 | ``` 169 | DEBUG Pushed block [blockId] in [time] ms 170 | ``` 171 | 172 | It then sends `AddBlock` (with `ReceivedBlockInfo` for `streamId`, `BlockStoreResult.numRecords`, `metadataOption`, and the result of `ReceivedBlockHandler.storeBlock`) to link:spark-streaming-receivertracker.adoc#ReceiverTrackerEndpoint[ReceiverTracker RPC endpoint] (that runs on the driver). 173 | 174 | When a response comes, you should see the following DEBUG message in the logs: 175 | 176 | ``` 177 | DEBUG Reported block [blockId] 178 | ``` 179 | -------------------------------------------------------------------------------- /spark-streaming-operators.adoc: -------------------------------------------------------------------------------- 1 | == Stream Operators 2 | 3 | You use *stream operators* to apply *transformations* to the elements received (often called *records*) from input streams and ultimately trigger computations using *output operators*. 4 | 5 | Transformations are *stateless*, but Spark Streaming comes with an _experimental_ support for link:spark-streaming-operators-stateful.adoc[stateful operators] (e.g. link:spark-streaming-operators-stateful.adoc#mapWithState[mapWithState] or link:spark-streaming-operators-stateful.adoc#updateStateByKey[updateStateByKey]). It also offers link:spark-streaming-windowedoperators.adoc[windowed operators] that can work across batches. 6 | 7 | NOTE: You may use RDDs from other (non-streaming) data sources to build more advanced pipelines. 8 | 9 | There are two main types of operators: 10 | 11 | * *transformations* that transform elements in input data RDDs 12 | * *output operators* that link:spark-streaming-dstreams.adoc#register[register input streams as output streams] so the execution can start. 13 | 14 | Every link:spark-streaming-dstreams.adoc[Discretized Stream (DStream)] offers the following operators: 15 | 16 | * (output operator) `print` to print 10 elements only or the more general version `print(num: Int)` to print up to `num` elements. See <> in this document. 17 | * link:spark-streaming-windowedoperators.adoc#slice[slice] 18 | * link:spark-streaming-windowedoperators.adoc#window[window] 19 | * link:spark-streaming-windowedoperators.adoc#reduceByWindow[reduceByWindow] 20 | * <> 21 | * <> 22 | * (output operator) <> 23 | * <> 24 | * (output operator) link:spark-streaming-operators-saveas.adoc[saveAsObjectFiles] 25 | * (output operator) link:spark-streaming-operators-saveas.adoc[saveAsTextFiles] 26 | * <> 27 | * <> 28 | * `flatMap` 29 | * `filter` 30 | * `repartition` 31 | * `mapPartitions` 32 | * `count` 33 | * `countByValue` 34 | * `countByWindow` 35 | * `countByValueAndWindow` 36 | * `union` 37 | 38 | NOTE: `DStream` companion object offers a Scala implicit to convert `DStream[(K, V)]` to `PairDStreamFunctions` with methods on DStreams of key-value pairs, e.g. link:spark-streaming-operators-stateful.adoc#mapWithState[mapWithState] or link:spark-streaming-operators-stateful.adoc#updateStateByKey[updateStateByKey]. 39 | 40 | Most streaming operators come with their own custom `DStream` to offer the service. It however very often boils down to overriding the link:spark-streaming-dstreams.adoc#contract[compute] method and applying corresponding link:spark-rdd-operations.adoc[RDD operator] on a generated RDD. 41 | 42 | === [[print]] print Operator 43 | 44 | `print(num: Int)` operator prints `num` first elements of each RDD in the input stream. 45 | 46 | `print` uses `print(num: Int)` with `num` being `10`. 47 | 48 | It is a *output operator* (that returns `Unit`). 49 | 50 | For each batch, `print` operator prints the following header to the standard output (regardless of the number of elements to be printed out): 51 | 52 | ``` 53 | ------------------------------------------- 54 | Time: [time] ms 55 | ------------------------------------------- 56 | ``` 57 | 58 | Internally, it calls `RDD.take(num + 1)` (see link:spark-rdd-actions.adoc[take action]) on each RDD in the stream to print `num` elements. It then prints `...` if there are more elements in the RDD (that would otherwise exceed `num` elements being requested to print). 59 | 60 | It creates a link:spark-streaming-foreachdstreams.adoc[ForEachDStream] stream and link:spark-streaming-dstreams.adoc#register[registers it as an output stream]. 61 | 62 | === [[foreachRDD]] foreachRDD Operators 63 | 64 | ``` 65 | foreachRDD(foreachFunc: RDD[T] => Unit): Unit 66 | foreachRDD(foreachFunc: (RDD[T], Time) => Unit): Unit 67 | ``` 68 | 69 | `foreachRDD` operator applies `foreachFunc` function to every RDD in the stream. 70 | 71 | It creates a link:spark-streaming-foreachdstreams.adoc[ForEachDStream] stream and link:spark-streaming-dstreams.adoc#register[registers it as an output stream]. 72 | 73 | ==== [[foreachRDD-example]] foreachRDD Example 74 | 75 | ``` 76 | val clicks: InputDStream[(String, String)] = messages 77 | // println every single data received in clicks input stream 78 | clicks.foreachRDD(rdd => rdd.foreach(println)) 79 | ``` 80 | 81 | === [[glom]] glom Operator 82 | 83 | ``` 84 | glom(): DStream[Array[T]] 85 | ``` 86 | 87 | `glom` operator creates a new stream in which RDDs in the source stream are link:spark-rdd-transformations.adoc[RDD.glom] over, i.e. it link:../spark-rdd-partitions.adoc#coalesce[coalesces] all elements in RDDs within each partition into an array. 88 | 89 | === [[reduce]] reduce Operator 90 | 91 | ``` 92 | reduce(reduceFunc: (T, T) => T): DStream[T] 93 | ``` 94 | 95 | `reduce` operator creates a new stream of RDDs of a single element that is a result of applying `reduceFunc` to the data received. 96 | 97 | Internally, it uses <> and <> operators. 98 | 99 | ==== [[reduce-example]] reduce Example 100 | 101 | [source, scala] 102 | ---- 103 | val clicks: InputDStream[(String, String)] = messages 104 | type T = (String, String) 105 | val reduceFunc: (T, T) => T = { 106 | case in @ ((k1, v1), (k2, v2)) => 107 | println(s">>> input: $in") 108 | (k2, s"$v1 + $v2") 109 | } 110 | val reduceClicks: DStream[(String, String)] = clicks.reduce(reduceFunc) 111 | reduceClicks.print 112 | ---- 113 | 114 | === [[map]] map Operator 115 | 116 | ``` 117 | map[U](mapFunc: T => U): DStream[U] 118 | ``` 119 | 120 | `map` operator creates a new stream with the source elements being mapped over using `mapFunc` function. 121 | 122 | It creates `MappedDStream` stream that, when requested to compute a RDD, uses link:spark-rdd-transformations.adoc[RDD.map] operator. 123 | 124 | ==== [[map-example]] map Example 125 | 126 | [source, scala] 127 | ---- 128 | val clicks: DStream[...] = ... 129 | val mappedClicks: ... = clicks.map(...) 130 | ---- 131 | 132 | === [[reduceByKey]] reduceByKey Operator 133 | 134 | [source, scala] 135 | ---- 136 | reduceByKey(reduceFunc: (V, V) => V): DStream[(K, V)] 137 | reduceByKey(reduceFunc: (V, V) => V, numPartitions: Int): DStream[(K, V)] 138 | reduceByKey(reduceFunc: (V, V) => V, partitioner: Partitioner): DStream[(K, V)] 139 | ---- 140 | 141 | === [[transform]] transform Operators 142 | 143 | ``` 144 | transform(transformFunc: RDD[T] => RDD[U]): DStream[U] 145 | transform(transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] 146 | ``` 147 | 148 | `transform` operator applies `transformFunc` function to the generated RDD for a batch. 149 | 150 | It creates a link:spark-streaming-transformeddstreams.adoc[TransformedDStream] stream. 151 | 152 | NOTE: It asserts that one and exactly one RDD has been generated for a batch before calling the `transformFunc`. 153 | 154 | NOTE: It is not allowed to return `null` from `transformFunc` or a `SparkException` is reported. See link:spark-streaming-transformeddstreams.adoc[TransformedDStream]. 155 | 156 | ==== [[transform-example]] transform Example 157 | 158 | ``` 159 | import org.apache.spark.streaming.{ StreamingContext, Seconds } 160 | val ssc = new StreamingContext(sc, batchDuration = Seconds(5)) 161 | 162 | val rdd = sc.parallelize(0 to 9) 163 | import org.apache.spark.streaming.dstream.ConstantInputDStream 164 | val clicks = new ConstantInputDStream(ssc, rdd) 165 | 166 | import org.apache.spark.rdd.RDD 167 | val transformFunc: RDD[Int] => RDD[Int] = { inputRDD => 168 | println(s">>> inputRDD: $inputRDD") 169 | 170 | // Use SparkSQL's DataFrame to manipulate the input records 171 | import spark.implicits._ 172 | inputRDD.toDF("num").show 173 | 174 | inputRDD 175 | } 176 | clicks.transform(transformFunc).print 177 | ``` 178 | 179 | === [[transformWith]] transformWith Operators 180 | 181 | ``` 182 | transformWith(other: DStream[U], transformFunc: (RDD[T], RDD[U]) => RDD[V]): DStream[V] 183 | transformWith(other: DStream[U], transformFunc: (RDD[T], RDD[U], Time) => RDD[V]): DStream[V] 184 | ``` 185 | 186 | `transformWith` operators apply the `transformFunc` function to two generated RDD for a batch. 187 | 188 | It creates a link:spark-streaming-transformeddstreams.adoc[TransformedDStream] stream. 189 | 190 | NOTE: It asserts that two and exactly two RDDs have been generated for a batch before calling the `transformFunc`. 191 | 192 | NOTE: It is not allowed to return `null` from `transformFunc` or a `SparkException` is reported. See link:spark-streaming-transformeddstreams.adoc[TransformedDStream]. 193 | 194 | ==== [[transformWith-example]] transformWith Example 195 | 196 | ``` 197 | import org.apache.spark.streaming.{ StreamingContext, Seconds } 198 | val ssc = new StreamingContext(sc, batchDuration = Seconds(5)) 199 | 200 | val ns = sc.parallelize(0 to 2) 201 | import org.apache.spark.streaming.dstream.ConstantInputDStream 202 | val nums = new ConstantInputDStream(ssc, ns) 203 | 204 | val ws = sc.parallelize(Seq("zero", "one", "two")) 205 | import org.apache.spark.streaming.dstream.ConstantInputDStream 206 | val words = new ConstantInputDStream(ssc, ws) 207 | 208 | import org.apache.spark.rdd.RDD 209 | import org.apache.spark.streaming.Time 210 | val transformFunc: (RDD[Int], RDD[String], Time) => RDD[(Int, String)] = { case (ns, ws, time) => 211 | println(s">>> ns: $ns") 212 | println(s">>> ws: $ws") 213 | println(s">>> batch: $time") 214 | 215 | ns.zip(ws) 216 | } 217 | nums.transformWith(words, transformFunc).print 218 | ``` 219 | -------------------------------------------------------------------------------- /spark-streaming-receivertracker.adoc: -------------------------------------------------------------------------------- 1 | == ReceiverTracker 2 | 3 | === [[introduction]] Introduction 4 | 5 | `ReceiverTracker` manages execution of all link:spark-streaming-receivers.adoc[Receivers]. 6 | 7 | .ReceiverTracker and Dependencies 8 | image::images/spark-streaming-receivertracker.png[align="center"] 9 | 10 | It uses link:spark-rpc.adoc[RPC environment] for communication with link:spark-streaming-receiversupervisors.adoc[ReceiverSupervisors]. 11 | 12 | NOTE: `ReceiverTracker` is started when link:spark-streaming-jobscheduler.adoc[JobScheduler] starts. 13 | 14 | It can only be started once and only when at least one input receiver has been registered. 15 | 16 | `ReceiverTracker` can be in one of the following states: 17 | 18 | * `Initialized` - it is in the state after having been instantiated. 19 | * `Started` - 20 | * `Stopping` 21 | * `Stopped` 22 | 23 | === [[starting]] Starting ReceiverTracker (start method) 24 | 25 | NOTE: You can only start `ReceiverTracker` once and multiple attempts lead to throwing `SparkException` exception. 26 | 27 | NOTE: Starting `ReceiverTracker` when no link:spark-streaming-receiverinputdstreams.adoc[ReceiverInputDStream] has registered does nothing. 28 | 29 | When `ReceiverTracker` starts, it first sets <> up. 30 | 31 | It then launches receivers, i.e. it collects receivers for all registered `ReceiverDStream` and posts them as <> to <>. 32 | 33 | In the meantime, receivers have their ids assigned that correspond to the unique identifier of their `ReceiverDStream`. 34 | 35 | You should see the following INFO message in the logs: 36 | 37 | ``` 38 | INFO ReceiverTracker: Starting [receivers.length] receivers 39 | ``` 40 | 41 | A successful startup of `ReceiverTracker` finishes with the following INFO message in the logs: 42 | 43 | ``` 44 | INFO ReceiverTracker: ReceiverTracker started 45 | ``` 46 | 47 | `ReceiverTracker` enters `Started` state. 48 | 49 | === [[cleanupOldBlocksAndBatches]] Cleanup Old Blocks And Batches (cleanupOldBlocksAndBatches method) 50 | 51 | CAUTION: FIXME 52 | 53 | === [[hasUnallocatedBlocks]] hasUnallocatedBlocks 54 | 55 | CAUTION: FIXME 56 | 57 | === [[ReceiverTrackerEndpoint]] ReceiverTracker RPC endpoint 58 | 59 | CAUTION: FIXME 60 | 61 | ==== [[ReceiverTrackerEndpoint-StartAllReceivers]] StartAllReceivers 62 | 63 | `StartAllReceivers(receivers)` is a local message sent by <> when <> (using `ReceiverTracker.launchReceivers()`). 64 | 65 | It schedules receivers (using `ReceiverSchedulingPolicy.scheduleReceivers(receivers, getExecutors)`). 66 | 67 | CAUTION: FIXME What does `ReceiverSchedulingPolicy.scheduleReceivers(receivers, getExecutors)` do? 68 | 69 | It does _some_ bookkeeping. 70 | 71 | CAUTION: FIXME What is _the_ bookkeeping? 72 | 73 | It finally starts every receiver (using the helper method <>). 74 | 75 | ===== [[ReceiverTrackerEndpoint-startReceiver]] ReceiverTrackerEndpoint.startReceiver 76 | 77 | CAUTION: FIXME When is the method called? 78 | 79 | `ReceiverTrackerEndpoint.startReceiver(receiver: Receiver[_], scheduledLocations: Seq[TaskLocation])` starts a `receiver` link:spark-streaming.adoc#Receiver[Receiver] at the given `Seq[TaskLocation]` locations. 80 | 81 | CAUTION: FIXME When the scaladoc says https://github.com/apache/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala#L543[_"along with the scheduled executors"_], does it mean that the executors are already started and waiting for the receiver?! 82 | 83 | It defines an internal function (`startReceiverFunc`) to start `receiver` on a worker (in Spark cluster). 84 | 85 | Namely, the internal `startReceiverFunc` function checks that the task attempt is `0`. 86 | 87 | TIP: Read about `TaskContext` in link:spark-taskscheduler-taskcontext.adoc[TaskContext]. 88 | 89 | It then starts a link:spark-streaming-receiversupervisors.adoc[ReceiverSupervisor] for `receiver` and keeps awaiting termination, i.e. once the task is run it does so until _a termination message_ comes from _some_ other external source). The task is a long-running task for `receiver`. 90 | 91 | CAUTION: FIXME When does `supervisor.awaitTermination()` finish? 92 | 93 | Having the internal function, it creates `receiverRDD` - an instance of `RDD[Receiver[_]]` - that uses link:spark-sparkcontext.adoc#makeRDD[SparkContext.makeRDD] with a one-element collection with the only element being `receiver`. When the collection of link:../spark-TaskLocation.adoc[TaskLocation] is empty, it uses exactly one partition. Otherwise, it distributes the one-element collection across the nodes (and potentially even executors) for `receiver`. The RDD has the name `Receiver [receiverId]`. 94 | 95 | The Spark job's description is set to `Streaming job running receiver [receiverId]`. 96 | 97 | CAUTION: FIXME What does `sparkContext.setJobDescription` actually do and how does this influence Spark jobs? It uses `ThreadLocal` so it assumes that a single thread will do a job? 98 | 99 | Having done so, it submits a job (using link:../spark-sparkcontext.adoc#submitJob[SparkContext.submitJob]) on the instance of `RDD[Receiver[_]]` with the function `startReceiverFunc` that runs `receiver`. It has link:../spark-rdd-actions.adoc#FutureAction[SimpleFutureAction] to monitor `receiver`. 100 | 101 | [NOTE] 102 | ==== 103 | The method demonstrates how you could use Spark Core as the distributed computation platform to launch _any_ process on clusters and let Spark handle the distribution. 104 | 105 | _Very clever indeed!_ 106 | ==== 107 | 108 | When it completes (successfully or not), `onReceiverJobFinish(receiverId)` is called, but only for cases when the tracker is fully up and running, i.e. started. When the tracker is being stopped or has already stopped, the following INFO message appears in the logs: 109 | 110 | ``` 111 | INFO Restarting Receiver [receiverId] 112 | ``` 113 | 114 | And a `RestartReceiver(receiver)` message is sent. 115 | 116 | When there was a failure submitting the job, you should also see the ERROR message in the logs: 117 | 118 | ``` 119 | ERROR Receiver has been stopped. Try to restart it. 120 | ``` 121 | 122 | Ultimately, right before the method exits, the following INFO message appears in the logs: 123 | 124 | ``` 125 | INFO Receiver [receiver.streamId] started 126 | ``` 127 | 128 | ==== [[ReceiverTrackerEndpoint-StopAllReceivers]] StopAllReceivers 129 | 130 | CAUTION: FIXME 131 | 132 | ==== [[ReceiverTrackerEndpoint-AllReceiverIds]] AllReceiverIds 133 | 134 | CAUTION: FIXME 135 | 136 | === [[stopping]] Stopping ReceiverTracker (stop method) 137 | 138 | `ReceiverTracker.stop(graceful: Boolean)` stops `ReceiverTracker` only when it is in `Started` state. Otherwise, it does nothing and simply exits. 139 | 140 | NOTE: The `stop` method is called while link:spark-streaming-jobscheduler.adoc#stopping[JobScheduler is being stopped]. 141 | 142 | The state of `ReceiverTracker` is marked `Stopping`. 143 | 144 | It then sends the stop signal to all the receivers (i.e. posts <> to <>) and waits *10 seconds* for all the receivers to quit gracefully (unless `graceful` flag is set). 145 | 146 | NOTE: The 10-second wait time for graceful quit is not configurable. 147 | 148 | You should see the following INFO messages if the `graceful` flag is enabled which means that the receivers quit in a graceful manner: 149 | 150 | ``` 151 | INFO ReceiverTracker: Waiting for receiver job to terminate gracefully 152 | INFO ReceiverTracker: Waited for receiver job to terminate gracefully 153 | ``` 154 | 155 | It then checks whether all the receivers have been deregistered or not by posting <> to <>. 156 | 157 | You should see the following INFO message in the logs if they have: 158 | 159 | ``` 160 | INFO ReceiverTracker: All of the receivers have deregistered successfully 161 | ``` 162 | 163 | Otherwise, when there were receivers not having been deregistered properly, the following WARN message appears in the logs: 164 | 165 | ``` 166 | WARN ReceiverTracker: Not all of the receivers have deregistered, [receivers] 167 | ``` 168 | 169 | It stops <> as well as <>. 170 | 171 | You should see the following INFO message in the logs: 172 | 173 | ``` 174 | INFO ReceiverTracker: ReceiverTracker stopped 175 | ``` 176 | 177 | The state of `ReceiverTracker` is marked `Stopped`. 178 | 179 | === [[allocateBlocksToBatch]] Allocating Blocks To Batch (allocateBlocksToBatch method) 180 | 181 | [source, scala] 182 | ---- 183 | allocateBlocksToBatch(batchTime: Time): Unit 184 | ---- 185 | 186 | `allocateBlocksToBatch` simply passes all the calls on to <>, but only when there _are_ link:spark-streaming-receiverinputdstreams.adoc[receiver input streams] registered (in `receiverInputStreams` internal registry). 187 | 188 | NOTE: When there are no link:spark-streaming-receiverinputdstreams.adoc[receiver input streams] in use, the method does nothing. 189 | 190 | === [[ReceivedBlockTracker]] ReceivedBlockTracker 191 | 192 | CAUTION: FIXME 193 | 194 | You should see the following INFO message in the logs when `cleanupOldBatches` is called: 195 | 196 | ``` 197 | INFO ReceivedBlockTracker: Deleting batches [timesToCleanup] 198 | ``` 199 | 200 | ==== [[ReceivedBlockTracker-allocateBlocksToBatch]] allocateBlocksToBatch Method 201 | 202 | [source, scala] 203 | ---- 204 | allocateBlocksToBatch(batchTime: Time): Unit 205 | ---- 206 | 207 | `allocateBlocksToBatch` starts by checking whether the internal `lastAllocatedBatchTime` is younger than (after) the current batch time `batchTime`. 208 | 209 | If so, it grabs all unallocated blocks per stream (using `getReceivedBlockQueue` method) and creates a map of stream ids and sequences of their `ReceivedBlockInfo`. It then writes the received blocks to *write-ahead log (WAL)* (using `writeToLog` method). 210 | 211 | `allocateBlocksToBatch` stores the allocated blocks with the current batch time in `timeToAllocatedBlocks` internal registry. It also sets `lastAllocatedBatchTime` to the current batch time `batchTime`. 212 | 213 | If there has been an error while writing to WAL or the batch time is older than `lastAllocatedBatchTime`, you should see the following INFO message in the logs: 214 | 215 | ``` 216 | INFO Possibly processed batch [batchTime] needs to be processed again in WAL recovery 217 | ``` 218 | -------------------------------------------------------------------------------- /spark-streaming-dstreamgraph.adoc: -------------------------------------------------------------------------------- 1 | == DStreamGraph 2 | 3 | `DStreamGraph` (is a final helper class that) manages *input* and *output dstreams*. It also holds <> for the other components that marks the time when <>. 4 | 5 | `DStreamGraph` maintains the collections of link:spark-streaming-inputdstreams.adoc[InputDStream] instances (as `inputStreams`) and output link:spark-streaming-dstreams.adoc[DStream] instances (as `outputStreams`), but, more importantly, <>. 6 | 7 | `DStreamGraph` holds the <> for the other parts of a Streaming application. 8 | 9 | [TIP] 10 | ==== 11 | Enable `INFO` or `DEBUG` logging level for `org.apache.spark.streaming.DStreamGraph` logger to see what happens in `DStreamGraph`. 12 | 13 | Add the following line to `conf/log4j.properties`: 14 | 15 | ``` 16 | log4j.logger.org.apache.spark.streaming.DStreamGraph=DEBUG 17 | ``` 18 | 19 | Refer to link:../spark-logging.adoc[Logging]. 20 | ==== 21 | 22 | === [[zero-time]][[zeroTime]] Zero Time (aka zeroTime) 23 | 24 | *Zero time* (internally `zeroTime`) is the time when <>. 25 | 26 | It is passed on down the output dstream graph so link:spark-streaming-dstreams.adoc#initialize[output dstreams can initialize themselves]. 27 | 28 | === [[startTime]] Start Time (aka startTime) 29 | 30 | *Start time* (internally `startTime`) is the time when <> or <>. 31 | 32 | NOTE: At regular start start time is exactly <>. 33 | 34 | === [[batchDuration]][[batch-interval]] Batch Interval (aka batchDuration) 35 | 36 | `DStreamGraph` holds the *batch interval* (as `batchDuration`) for the other parts of a Streaming application. 37 | 38 | `setBatchDuration(duration: Duration)` is the method to set the batch interval. 39 | 40 | It appears that it is _the_ place for the value since it must be set before link:spark-streaming-jobgenerator.adoc[JobGenerator] can be instantiated. 41 | 42 | It _is_ set while link:spark-streaming-streamingcontext.adoc[StreamingContext] is being instantiated and is validated (using `validate()` method of `StreamingContext` and `DStreamGraph`) before `StreamingContext` is started. 43 | 44 | === [[getMaxInputStreamRememberDuration]][[maximum-remember-interval]] Maximum Remember Interval -- getMaxInputStreamRememberDuration Method 45 | 46 | [source, scala] 47 | ---- 48 | getMaxInputStreamRememberDuration(): Duration 49 | ---- 50 | 51 | *Maximum Remember Interval* is the maximum link:spark-streaming-dstreams.adoc#remember-interval[remember interval] across all the input dstreams. It is calculated using `getMaxInputStreamRememberDuration` method. 52 | 53 | NOTE: It is called when JobGenerator is requested to link:spark-streaming-jobgenerator.adoc#clearMetadata[clear metadata] and link:spark-streaming-jobgenerator.adoc#clearCheckpointData[checkpoint data]. 54 | 55 | === [[input-dstream-registry]] Input DStreams Registry 56 | 57 | CAUTION: FIXME 58 | 59 | === [[output-dstreams]][[output-dstream-registry]] Output DStreams Registry 60 | 61 | `DStream` by design has no notion of being an output dstream. To mark a dstream as output you need to link:spark-streaming-dstreams.adoc#register[register a dstream (using DStream.register method)] which happens for...FIXME 62 | 63 | === [[start]] Starting `DStreamGraph` 64 | 65 | [source, scala] 66 | ---- 67 | start(time: Time): Unit 68 | ---- 69 | 70 | When `DStreamGraph` is started (using `start` method), it sets <> and <>. 71 | 72 | NOTE: `start` method is called when link:spark-streaming-jobgenerator.adoc#starting[JobGenerator starts for the first time] (not from a checkpoint). 73 | 74 | NOTE: You can start `DStreamGraph` as many times until `time` is not `null` and <> has been set. 75 | 76 | (_output dstreams_) `start` then walks over the collection of output dstreams and for each output dstream, one at a time, calls their link:spark-streaming-dstreams.adoc#initialize[initialize(zeroTime)], link:spark-streaming-dstreams.adoc#remember[remember] (with the current <>), and link:spark-streaming-dstreams.adoc#validateAtStart[validateAtStart] methods. 77 | 78 | (_input dstreams_) When all the output streams are processed, it starts the input dstreams (in parallel) using `start` method. 79 | 80 | === [[stop]] Stopping `DStreamGraph` 81 | 82 | [source, scala] 83 | ---- 84 | stop(): Unit 85 | ---- 86 | 87 | CAUTION: FIXME 88 | 89 | === [[restart]] Restarting `DStreamGraph` 90 | 91 | [source, scala] 92 | ---- 93 | restart(time: Time): Unit 94 | ---- 95 | 96 | `restart` sets <> to be `time` input parameter. 97 | 98 | NOTE: This is the only moment when <> can be different than <>. 99 | 100 | CAUTION: `restart` doesn't seem to be called ever. 101 | 102 | === [[generateJobs]] Generating Streaming Jobs for Output DStreams for Batch Time -- `generateJobs` Method 103 | 104 | [source, scala] 105 | ---- 106 | generateJobs(time: Time): Seq[Job] 107 | ---- 108 | 109 | `generateJobs` method generates a collection of streaming jobs for output streams for a given batch `time`. It walks over each link:spark-streaming-dstreams.adoc#register[registered output stream] (in `outputStreams` internal registry) and link:spark-streaming-dstreams.adoc#generateJob[requests each stream for a streaming job] 110 | 111 | NOTE: `generateJobs` is called by link:spark-streaming-jobgenerator.adoc[JobGenerator] to link:spark-streaming-jobgenerator.adoc#generateJobs[generate jobs for a given batch time] or link:spark-streaming-jobgenerator.adoc#restarting[when restarted from checkpoint]. 112 | 113 | When `generateJobs` method executes, you should see the following DEBUG message in the logs: 114 | 115 | ``` 116 | DEBUG DStreamGraph: Generating jobs for time [time] ms 117 | ``` 118 | 119 | `generateJobs` then walks over each link:spark-streaming-dstreams.adoc#register[registered output stream] (in `outputStreams` internal registry) and link:spark-streaming-dstreams.adoc#generateJob[requests the streams for a streaming job]. 120 | 121 | Right before the method finishes, you should see the following DEBUG message with the number of streaming jobs generated (as `jobs.length`): 122 | 123 | ``` 124 | DEBUG DStreamGraph: Generated [jobs.length] jobs for time [time] ms 125 | ``` 126 | 127 | === [[validate]][[dstreamgraph-validation]] Validation Check 128 | 129 | `validate()` method checks whether batch duration and at least one output stream have been set. It will throw `java.lang.IllegalArgumentException` when either is not. 130 | 131 | NOTE: It is called when link:spark-streaming-streamingcontext.adoc#start[StreamingContext starts]. 132 | 133 | === [[clearMetadata]] Metadata Cleanup 134 | 135 | NOTE: It is called when link:spark-streaming-jobgenerator.adoc#ClearMetadata[JobGenerator clears metadata]. 136 | 137 | When `clearMetadata(time: Time)` is called, you should see the following DEBUG message in the logs: 138 | 139 | ``` 140 | DEBUG DStreamGraph: Clearing metadata for time [time] ms 141 | ``` 142 | 143 | It merely walks over the collection of output streams and (synchronously, one by one) asks to do link:spark-streaming-dstreams.adoc#clearMetadata[its own metadata cleaning]. 144 | 145 | When finishes, you should see the following DEBUG message in the logs: 146 | 147 | ``` 148 | DEBUG DStreamGraph: Cleared old metadata for time [time] ms 149 | ``` 150 | 151 | === [[restoreCheckpointData]] Restoring State for Output DStreams -- `restoreCheckpointData` Method 152 | 153 | [source, scala] 154 | ---- 155 | restoreCheckpointData(): Unit 156 | ---- 157 | 158 | When `restoreCheckpointData()` is executed, you should see the following INFO message in the logs: 159 | 160 | ``` 161 | INFO DStreamGraph: Restoring checkpoint data 162 | ``` 163 | 164 | Then, every <> is requested to link:spark-streaming-dstreams.adoc#restoreCheckpointData[restoreCheckpointData]. 165 | 166 | At the end, you should see the following INFO message in the logs: 167 | 168 | ``` 169 | INFO DStreamGraph: Restored checkpoint data 170 | ``` 171 | 172 | NOTE: `restoreCheckpointData` is executed when link:spark-streaming-checkpointing.adoc#recreating-streamingcontext[StreamingContext is recreated from checkpoint]. 173 | 174 | === [[updateCheckpointData]] Updating Checkpoint Data -- `updateCheckpointData` Method 175 | 176 | [source, scala] 177 | ---- 178 | updateCheckpointData(time: Time): Unit 179 | ---- 180 | 181 | NOTE: `updateCheckpointData` is called when link:spark-streaming-jobgenerator.adoc#DoCheckpoint[JobGenerator processes DoCheckpoint events]. 182 | 183 | When `updateCheckpointData` is called, you should see the following INFO message in the logs: 184 | 185 | ``` 186 | INFO DStreamGraph: Updating checkpoint data for time [time] ms 187 | ``` 188 | 189 | It then walks over every output dstream and calls its link:spark-streaming-dstreams.adoc#updateCheckpointData[updateCheckpointData(time)]. 190 | 191 | When `updateCheckpointData` finishes it prints out the following INFO message to the logs: 192 | 193 | ``` 194 | INFO DStreamGraph: Updated checkpoint data for time [time] ms 195 | ``` 196 | 197 | === [[clearCheckpointData]] Checkpoint Cleanup -- `clearCheckpointData` Method 198 | 199 | [source, scala] 200 | ---- 201 | clearCheckpointData(time: Time) 202 | ---- 203 | 204 | NOTE: `clearCheckpointData` is called when link:spark-streaming-jobgenerator.adoc#clearCheckpointData[JobGenerator clears checkpoint data]. 205 | 206 | When `clearCheckpointData` is called, you should see the following INFO message in the logs: 207 | 208 | ``` 209 | INFO DStreamGraph: Clearing checkpoint data for time [time] ms 210 | ``` 211 | 212 | It merely walks through the collection of output streams and (synchronously, one by one) asks to do link:spark-streaming-dstreams.adoc#clearCheckpointData[their own checkpoint data cleaning]. 213 | 214 | When finished, you should see the following INFO message in the logs: 215 | 216 | ``` 217 | INFO DStreamGraph: Cleared checkpoint data for time [time] ms 218 | ``` 219 | 220 | === [[rememberDuration]][[remember-interval]] Remember Interval 221 | 222 | *Remember interval* is the time to remember (aka _cache_) the RDDs that have been generated by (output) dstreams in the context (before they are released and garbage collected). 223 | 224 | It can be set using <> method. 225 | 226 | === [[remember]] `remember` Method 227 | 228 | [source, scala] 229 | ---- 230 | remember(duration: Duration): Unit 231 | ---- 232 | 233 | `remember` method simply sets <> and exits. 234 | 235 | NOTE: It is called by link:spark-streaming-streamingcontext.adoc#remember[StreamingContext.remember] method. 236 | 237 | It first checks whether or not it has been set already and if so, throws `java.lang.IllegalArgumentException` as follows: 238 | 239 | [options="wrap"] 240 | ---- 241 | java.lang.IllegalArgumentException: requirement failed: Remember duration already set as [rememberDuration] ms. Cannot set it again. 242 | at scala.Predef$.require(Predef.scala:219) 243 | at org.apache.spark.streaming.DStreamGraph.remember(DStreamGraph.scala:79) 244 | at org.apache.spark.streaming.StreamingContext.remember(StreamingContext.scala:222) 245 | ... 43 elided 246 | ---- 247 | 248 | NOTE: It only makes sense to call `remember` method before <>, i.e. before link:spark-streaming-streamingcontext.adoc#start[StreamingContext is started], since the output dstreams are only given the remember interval when DStreamGraph starts. 249 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "{}" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright {yyyy} {name of copyright owner} 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. 202 | -------------------------------------------------------------------------------- /spark-streaming-jobscheduler.adoc: -------------------------------------------------------------------------------- 1 | == JobScheduler 2 | 3 | *Streaming scheduler* (`JobScheduler`) schedules streaming jobs to be run as Spark jobs. It is created as part of link:spark-streaming-streamingcontext.adoc#creating-instance[creating a StreamingContext] and starts with it. 4 | 5 | .JobScheduler and Dependencies 6 | image::images/spark-streaming-jobscheduler.png[align="center"] 7 | 8 | It tracks jobs submitted for execution (as <> via <> method) in <> internal map. 9 | 10 | NOTE: JobSets are submitted by link:spark-streaming-jobgenerator.adoc[JobGenerator]. 11 | 12 | It uses a *streaming scheduler queue* for streaming jobs to be executed. 13 | 14 | [TIP] 15 | ==== 16 | Enable `DEBUG` logging level for `org.apache.spark.streaming.scheduler.JobScheduler` logger to see what happens in JobScheduler. 17 | 18 | Add the following line to `conf/log4j.properties`: 19 | 20 | ``` 21 | log4j.logger.org.apache.spark.streaming.scheduler.JobScheduler=DEBUG 22 | ``` 23 | 24 | Refer to link:../spark-logging.adoc[Logging]. 25 | ==== 26 | 27 | === [[start]][[starting]] Starting JobScheduler (start method) 28 | 29 | [source, scala] 30 | ---- 31 | start(): Unit 32 | ---- 33 | 34 | When `JobScheduler` starts (i.e. when `start` is called), you should see the following DEBUG message in the logs: 35 | 36 | ``` 37 | DEBUG JobScheduler: Starting JobScheduler 38 | ``` 39 | 40 | It then goes over all the dependent services and starts them one by one as depicted in the figure. 41 | 42 | .JobScheduler Start procedure 43 | image::images/spark-streaming-JobScheduler-start.png[align="center"] 44 | 45 | It first starts <>. 46 | 47 | It asks link:spark-streaming-dstreamgraph.adoc#input-dstream-registry[DStreamGraph for input dstreams] and registers link:spark-streaming-backpressure.adoc#RateController[their RateControllers] (if defined) as link:spark-streaming-streamingcontext.adoc#addStreamingListener[streaming listeners]. It starts <> afterwards. 48 | 49 | It creates link:spark-streaming-receivertracker.adoc[ReceiverTracker] and link:spark-streaming-InputInfoTracker.adoc[InputInfoTracker]. It then starts the `ReceiverTracker`. 50 | 51 | It starts link:spark-streaming-jobgenerator.adoc[JobGenerator]. 52 | 53 | Just before `start` finishes, you should see the following INFO message in the logs: 54 | 55 | ``` 56 | INFO JobScheduler: Started JobScheduler 57 | ``` 58 | 59 | === [[getPendingTimes]] Pending Batches to Process (getPendingTimes method) 60 | 61 | CAUTION: FIXME 62 | 63 | === [[stop]][[stopping]] Stopping JobScheduler (stop method) 64 | 65 | [source, scala] 66 | ---- 67 | stop(processAllReceivedData: Boolean): Unit 68 | ---- 69 | 70 | `stop` stops `JobScheduler`. 71 | 72 | NOTE: It is called when link:spark-streaming-streamingcontext.adoc#stopping[StreamingContext is being stopped]. 73 | 74 | You should see the following DEBUG message in the logs: 75 | 76 | ``` 77 | DEBUG JobScheduler: Stopping JobScheduler 78 | ``` 79 | 80 | link:spark-streaming-receivertracker.adoc#stopping[ReceiverTracker is stopped]. 81 | 82 | NOTE: link:spark-streaming-receivertracker.adoc[ReceiverTracker] is only assigned (and started) while JobScheduler is starting. 83 | 84 | It link:spark-streaming-jobgenerator.adoc#stop[stops generating jobs]. 85 | 86 | You should see the following DEBUG message in the logs: 87 | 88 | ``` 89 | DEBUG JobScheduler: Stopping job executor 90 | ``` 91 | 92 | <> is shut down (using `jobExecutor.shutdown()`). 93 | 94 | If the stop should wait for all received data to be processed (the input parameter `processAllReceivedData` is `true`), `stop` awaits termination of <> for *1 hour* (it is assumed that it is enough and is not configurable). Otherwise, it waits for *2 seconds*. 95 | 96 | <> is forcefully shut down (using `jobExecutor.shutdownNow()`) unless it has terminated already. 97 | 98 | You should see the following DEBUG message in the logs: 99 | 100 | ``` 101 | DEBUG JobScheduler: Stopped job executor 102 | ``` 103 | 104 | <> and <> are stopped. 105 | 106 | You should see the following INFO message in the logs: 107 | 108 | ``` 109 | INFO JobScheduler: Stopped JobScheduler 110 | ``` 111 | 112 | === [[submitJobSet]] Submitting Collection of Jobs for Execution -- `submitJobSet` method 113 | 114 | When `submitJobSet(jobSet: JobSet)` is called, it reacts appropriately per `jobSet` <> given. 115 | 116 | NOTE: The method is called by link:spark-streaming-jobgenerator.adoc[JobGenerator] only (as part of link:spark-streaming-jobgenerator.adoc#generateJobs[JobGenerator.generateJobs] and link:spark-streaming-jobgenerator.adoc#restarting[JobGenerator.restart]). 117 | 118 | When no streaming jobs are inside the `jobSet`, you should see the following INFO in the logs: 119 | 120 | ``` 121 | INFO JobScheduler: No jobs added for time [jobSet.time] 122 | ``` 123 | 124 | Otherwise, when there is at least one streaming job inside the `jobSet`, link:spark-streaming-streaminglisteners.adoc#StreamingListenerEvent[StreamingListenerBatchSubmitted] (with data statistics of every registered input stream for which the streaming jobs were generated) is posted to <>. 125 | 126 | The JobSet is added to the internal <> registry. 127 | 128 | It then goes over every streaming job in the `jobSet` and executes a <> (on <>). 129 | 130 | At the end, you should see the following INFO message in the logs: 131 | 132 | ``` 133 | INFO JobScheduler: Added jobs for time [jobSet.time] ms 134 | ``` 135 | 136 | === [[JobHandler]] JobHandler 137 | 138 | `JobHandler` is a thread of execution for a link:spark-streaming.adoc#Job[streaming job] (that simply calls `Job.run`). 139 | 140 | NOTE: It is called when a new <> is submitted (see <> in this document). 141 | 142 | When started, it prepares the environment (so the streaming job can be nicely displayed in the web UI under `/streaming/batch/?id=[milliseconds]`) and posts `JobStarted` event to <>. 143 | 144 | It runs the link:spark-streaming.adoc#Job[streaming job] that executes the job function as defined while link:spark-streaming-dstreams.adoc#generateJob[generating a streaming job for an output stream]. 145 | 146 | NOTE: This is when Spark is requested to link:../spark-sparkcontext.adoc#runJob[run a Spark job]. 147 | 148 | You may see similar-looking INFO messages in the logs (it depends on the link:spark-streaming-operators.adoc[operators] you use): 149 | 150 | ``` 151 | INFO SparkContext: Starting job: print at :39 152 | INFO DAGScheduler: Got job 0 (print at :39) with 1 output partitions 153 | ... 154 | INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (KafkaRDD[2] at createDirectStream at :36) 155 | ... 156 | INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 987 bytes result sent to driver 157 | ... 158 | INFO DAGScheduler: Job 0 finished: print at :39, took 0.178689 s 159 | ``` 160 | 161 | It posts `JobCompleted` event to <>. 162 | 163 | === [[streaming-job-executor]] jobExecutor Thread Pool 164 | 165 | While `JobScheduler` is instantiated, the daemon thread pool `streaming-job-executor-ID` with link:spark-streaming-settings.adoc[spark.streaming.concurrentJobs] threads is created. 166 | 167 | It is used to execute <> for jobs in JobSet (see <> in this document). 168 | 169 | It shuts down when link:spark-streaming-streamingcontext.adoc#stop[StreamingContext] stops. 170 | 171 | === [[eventLoop]] eventLoop - JobSchedulerEvent Handler 172 | 173 | JobScheduler uses `EventLoop` for `JobSchedulerEvent` events. It accepts <> and <> events. It also processes `ErrorReported` events. 174 | 175 | ==== [[JobStarted]] JobStarted and JobScheduler.handleJobStart 176 | 177 | When `JobStarted` event is received, `JobScheduler.handleJobStart` is called. 178 | 179 | NOTE: It is <> to post `JobStarted`. 180 | 181 | `handleJobStart(job: Job, startTime: Long)` takes a `JobSet` (from `jobSets`) and checks whether it has already been started. 182 | 183 | It posts `StreamingListenerBatchStarted` to <> when the JobSet is about to start. 184 | 185 | It posts `StreamingListenerOutputOperationStarted` to <>. 186 | 187 | You should see the following INFO message in the logs: 188 | 189 | ``` 190 | INFO JobScheduler: Starting job [job.id] from job set of time [jobSet.time] ms 191 | ``` 192 | 193 | ==== [[JobCompleted]] JobCompleted and JobScheduler.handleJobCompletion 194 | 195 | When `JobCompleted` event is received, it triggers `JobScheduler.handleJobCompletion(job: Job, completedTime: Long)`. 196 | 197 | NOTE: <> posts `JobCompleted` events when it finishes running a streaming job. 198 | 199 | `handleJobCompletion` looks the <> up (from the <> internal registry) and calls <> (that marks the `JobSet` as completed when no more streaming jobs are incomplete). It also calls `Job.setEndTime(completedTime)`. 200 | 201 | It posts `StreamingListenerOutputOperationCompleted` to <>. 202 | 203 | You should see the following INFO message in the logs: 204 | 205 | ``` 206 | INFO JobScheduler: Finished job [job.id] from job set of time [jobSet.time] ms 207 | ``` 208 | 209 | If the entire JobSet is completed, it removes it from <>, and calls link:spark-streaming-jobgenerator.adoc#onBatchCompletion[JobGenerator.onBatchCompletion]. 210 | 211 | You should see the following INFO message in the logs: 212 | 213 | ``` 214 | INFO JobScheduler: Total delay: [totalDelay] s for time [time] ms (execution: [processingDelay] s) 215 | ``` 216 | 217 | It posts `StreamingListenerBatchCompleted` to <>. 218 | 219 | It reports an error if the job's result is a failure. 220 | 221 | === [[StreamingListenerBus]] StreamingListenerBus and StreamingListenerEvents 222 | 223 | `StreamingListenerBus` is a asynchronous listener bus to post `StreamingListenerEvent` events to link:spark-streaming-streaminglisteners.adoc[streaming listeners]. 224 | 225 | === [[internal-registries]] Internal Registries 226 | 227 | `JobScheduler` maintains the following information in internal registries: 228 | 229 | * `jobSets` - a mapping between time and JobSets. See <>. 230 | 231 | === [[JobSet]] JobSet 232 | 233 | A `JobSet` represents a collection of link:spark-streaming.adoc#Job[streaming jobs] that were created at (batch) `time` for link:spark-streaming-dstreamgraph.adoc#generateJobs[output streams] (that have ultimately produced a streaming job as they may opt out). 234 | 235 | .JobSet Created and Submitted to JobScheduler 236 | image::images/spark-streaming-jobset-generatejobs-event.png[align="center"] 237 | 238 | `JobSet` tracks what streaming jobs are in incomplete state (in `incompleteJobs` internal registry). 239 | 240 | NOTE: At the beginning (when `JobSet` is created) all streaming jobs are incomplete. 241 | 242 | CAUTION: FIXME There is a duplication in how streaming jobs are tracked as completed since a `Job` knows about its `_endTime`. Is this a optimization? How much time does it buy us? 243 | 244 | A `JobSet` tracks the following moments in its lifecycle: 245 | 246 | * `submissionTime` being the time when the instance was created. 247 | 248 | * `processingStartTime` being the time when the first streaming job in the collection was started. 249 | 250 | * `processingEndTime` being the time when the last streaming job in the collection finished processing. 251 | 252 | A `JobSet` changes state over time. It can be in the following states: 253 | 254 | * *Created* after a `JobSet` was created. `submissionTime` is set. 255 | * *Started* after `JobSet.handleJobStart` was called. `processingStartTime` is set. 256 | * *Completed* after `JobSet.handleJobCompletion` and no more jobs are incomplete (in `incompleteJobs` internal registry). `processingEndTime` is set. 257 | 258 | .JobSet States 259 | image::images/spark-streaming-jobset-states.png[align="center"] 260 | 261 | Given the states a `JobSet` has *delays*: 262 | 263 | * *Processing delay* is the time spent for processing all the streaming jobs in a `JobSet` from the time the very first job was started, i.e. the time between started and completed states. 264 | 265 | * *Total delay* is the time from the batch time until the `JobSet` was completed. 266 | 267 | NOTE: Total delay is always longer than processing delay. 268 | 269 | You can map a `JobSet` to a `BatchInfo` using `toBatchInfo` method. 270 | 271 | NOTE: `BatchInfo` is used to create and post link:spark-streaming-streaminglisteners.adoc#StreamingListenerEvent[StreamingListenerBatchSubmitted], link:spark-streaming-streaminglisteners.adoc#StreamingListenerEvent[StreamingListenerBatchStarted], and link:spark-streaming-streaminglisteners.adoc#StreamingListenerEvent[StreamingListenerBatchCompleted] events. 272 | 273 | `JobSet` is used (created or processed) in: 274 | 275 | * link:spark-streaming-jobgenerator.adoc#generateJobs[JobGenerator.generateJobs] 276 | * <> 277 | * link:spark-streaming-jobgenerator.adoc#restarting[JobGenerator.restart] 278 | * <> 279 | * <> 280 | -------------------------------------------------------------------------------- /spark-streaming-kafka-DirectKafkaInputDStream.adoc: -------------------------------------------------------------------------------- 1 | == [[DirectKafkaInputDStream]] DirectKafkaInputDStream -- Direct Kafka DStream 2 | 3 | `DirectKafkaInputDStream` is an link:spark-streaming-inputdstreams.adoc[input dstream] of link:spark-streaming-kafka-KafkaRDD.adoc[KafkaRDD] batches. 4 | 5 | `DirectKafkaInputDStream` is also a `CanCommitOffsets` object. 6 | 7 | As an input dstream, `DirectKafkaInputDStream` implements the mandatory abstract Methods (from link:spark-streaming-dstreams.adoc#contract[DStream Contract] and link:spark-streaming-inputdstreams.adoc#contract[InputDStream Contract]): 8 | 9 | 1. `dependencies` returns an empty collection, i.e. it has no dependencies on other streams (other than Kafka brokers to read data from). 10 | 2. `slideDuration` passes all calls on to link:spark-streaming-dstreamgraph.adoc[DStreamGraph.batchDuration]. 11 | 3. <> to create a `KafkaRDD` per batch. 12 | 4. <> to start polling for messages from Kafka. 13 | 5. <> to close the Kafka consumer (and therefore polling for messages from Kafka). 14 | 15 | The `name` of a `DirectKafkaInputDStream` is *Kafka 0.10 direct stream [id]* (that you can use to differentiate between the different implementations for Kafka 0.10+ and older releases). 16 | 17 | TIP: You can find the name of a input dstream in the link:spark-streaming-webui.adoc[Streaming tab] in web UI (in the details of a batch in *Input Metadata* section). 18 | 19 | It uses link:spark-streaming-settings.adoc[spark.streaming.kafka.maxRetries] setting while computing `latestLeaderOffsets` (i.e. a mapping of `kafka.common.TopicAndPartition` and <>). 20 | 21 | [TIP] 22 | ==== 23 | Enable `INFO` logging level for `org.apache.spark.streaming.kafka010.DirectKafkaInputDStream` logger to see what happens inside. 24 | 25 | Add the following line to `conf/log4j.properties`: 26 | 27 | ``` 28 | log4j.logger.org.apache.spark.streaming.kafka010.DirectKafkaInputDStream=INFO 29 | ``` 30 | 31 | Refer to link:../spark-logging.adoc[Logging]. 32 | ==== 33 | 34 | === [[creating-instance]] Creating `DirectKafkaInputDStream` Instance 35 | 36 | You can create a `DirectKafkaInputDStream` instance using link:spark-streaming-kafka-KafkaUtils.adoc#createDirectStream[KafkaUtils.createDirectStream] factory method. 37 | 38 | [source, scala] 39 | ---- 40 | import org.apache.spark.streaming.kafka010.KafkaUtils 41 | 42 | // WARN: Incomplete to show only relevant parts 43 | val dstream = KafkaUtils.createDirectStream[String, String]( 44 | ssc = streamingContext, 45 | locationStrategy = hosts, 46 | consumerStrategy = ConsumerStrategies.Subscribe[String, String](topics, kafkaParams, offsets)) 47 | ---- 48 | 49 | Internally, when a `DirectKafkaInputDStream` instance is created, it initializes the internal <> using the input ``consumerStrategy``'s link:spark-streaming-kafka-ConsumerStrategy.adoc#executorKafkaParams[executorKafkaParams]. 50 | 51 | TIP: Use link:spark-streaming-kafka-ConsumerStrategy.adoc[ConsumerStrategy] for a Kafka Consumer configuration. 52 | 53 | With link:spark-streaming-kafka-KafkaUtils.adoc#logging[WARN logging level enabled for the KafkaUtils logger], you may see the following WARN messages and one ERROR in the logs (the number of messages depends on how correct the Kafka Consumer configuration is): 54 | 55 | ``` 56 | WARN KafkaUtils: overriding enable.auto.commit to false for executor 57 | WARN KafkaUtils: overriding auto.offset.reset to none for executor 58 | ERROR KafkaUtils: group.id is null, you should probably set it 59 | WARN KafkaUtils: overriding executor group.id to spark-executor-null 60 | WARN KafkaUtils: overriding receive.buffer.bytes to 65536 see KAFKA-3135 61 | ``` 62 | 63 | [TIP] 64 | ==== 65 | You should always set `group.id` in Kafka parameters for `DirectKafkaInputDStream`. 66 | 67 | Refer to link:spark-streaming-kafka-ConsumerStrategy.adoc[ConsumerStrategy -- Kafka Consumers' Post-Configuration API]. 68 | ==== 69 | 70 | It initializes the internal <> property. 71 | 72 | It creates an instance of `DirectKafkaInputDStreamCheckpointData` as `checkpointData`. 73 | 74 | It sets up `rateController` as `DirectKafkaRateController` when backpressure is enabled. 75 | 76 | It sets up `maxRateLimitPerPartition` as link:spark-streaming-settings.adoc#spark_streaming_kafka_maxRatePerPartition[spark.streaming.kafka.maxRatePerPartition]. 77 | 78 | It initializes <> and <> properties. 79 | 80 | === [[currentOffsets]] `currentOffsets` Property 81 | 82 | [source, scala] 83 | ---- 84 | currentOffsets: Map[TopicPartition, Long] 85 | ---- 86 | 87 | `currentOffsets` holds the latest (highest) available offsets for all the topic partitions the dstream is subscribed to (as set by <> and <>). 88 | 89 | `currentOffsets` is initialized when <> (it could also be re-created from a checkpoint). 90 | 91 | The link:spark-streaming-kafka-ConsumerStrategy.adoc#onStart[ConsumerStrategy] (that was used to initialize `DirectKafkaInputDStream`) uses it to <>. 92 | 93 | It is then set to the available offsets when <>. 94 | 95 | === [[commitCallback]] `commitCallback` Property 96 | 97 | [source, scala] 98 | ---- 99 | commitCallback: AtomicReference[OffsetCommitCallback] 100 | ---- 101 | 102 | `commitCallback` is initialized when <>. It is set to a `OffsetCommitCallback` that is the input parameter of `commitAsync` when it is called (as part of the `CanCommitOffsets` contract that `DirectKafkaInputDStream` implements). 103 | 104 | === [[commitQueue]] `commitQueue` Property 105 | 106 | [source, scala] 107 | ---- 108 | commitQueue: ConcurrentLinkedQueue[OffsetRange] 109 | ---- 110 | 111 | `commitQueue` is initialized when <>. It is used in `commitAsync` (that is part of the `CanCommitOffsets` contract that `DirectKafkaInputDStream` implements) to queue up offsets for commit to Kafka at a future time (i.e. when the internal <> is called). 112 | 113 | TIP: Read https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ConcurrentLinkedQueue.html[java.util.concurrent.ConcurrentLinkedQueue] javadoc. 114 | 115 | === [[executorKafkaParams]] `executorKafkaParams` Attribute 116 | 117 | [source, scala] 118 | ---- 119 | executorKafkaParams: HashMap[String, Object] 120 | ---- 121 | 122 | `executorKafkaParams` is a collection of ...FIXME 123 | 124 | When <>, it initializes `executorKafkaParams` with link:spark-streaming-kafka-ConsumerStrategy.adoc#executorKafkaParams[`executorKafkaParams` of the given `ConsumerStrategy`] (that was used to create the `DirectKafkaInputDStream` instance). 125 | 126 | `executorKafkaParams` is then link:spark-streaming-kafka-KafkaUtils.adoc#fixKafkaParams[reviewed and corrected where needed]. 127 | 128 | NOTE: `executorKafkaParams` is used when <> and restoring ``KafkaRDD``s from checkpoint. 129 | 130 | === [[start]] Starting `DirectKafkaInputDStream` -- `start` Method 131 | 132 | [source, scala] 133 | ---- 134 | start(): Unit 135 | ---- 136 | 137 | `start` creates a <> and fetches available records in the subscribed list of topics and partitions (using Kafka's link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/Consumer.html#poll(long)++[Consumer.poll] with `0` timeout that says to return immediately with any records that are available currently). 138 | 139 | NOTE: `start` is part of the link:spark-streaming-inputdstreams.adoc[InputDStream Contract]. 140 | 141 | After the polling, `start` checks if the internal <> is empty, and if it is, it requests Kafka for topic (using Kafka's link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/Consumer.html#assignment()++[Consumer.assignment]) and builds a map with topics and their offsets (using Kafka's link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/Consumer.html#position(org.apache.kafka.common.TopicPartition)++[Consumer.position]). 142 | 143 | Ultimately, `start` pauses all partitions (using Kafka's link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/Consumer.html#pause(java.util.Collection)++[Consumer.pause] with the internal collection of topics and their current offsets). 144 | 145 | === [[compute]] Generating KafkaRDD for Batch Interval -- `compute` Method 146 | 147 | [source, scala] 148 | ---- 149 | compute(validTime: Time): Option[KafkaRDD[K, V]] 150 | ---- 151 | 152 | NOTE: `compute` is a part of the link:spark-streaming-dstreams.adoc[DStream Contract]. 153 | 154 | `compute` _always_ computes a link:spark-streaming-kafka-KafkaRDD.adoc[KafkaRDD] (despite the return type that allows for no RDDs and irrespective the number of records inside). It is left to a `KafkaRDD` itself to decide what to do when no Kafka records exist in topic partitions to process for a given batch. 155 | 156 | NOTE: It is link:spark-streaming-dstreamgraph.adoc#generateJobs[`DStreamGraph` to request generating streaming jobs for batches]. 157 | 158 | When `compute` is called, it calls <> and <>. The result topic partition offsets are then mapped to link:spark-streaming-kafka-HasOffsetRanges.adoc#OffsetRange[OffsetRange]s with a topic, a partition, and <> and the result offset. That in turn is used to create link:spark-streaming-kafka-KafkaRDD.adoc[KafkaRDD] (with the current link:../spark-sparkcontext.adoc[SparkContext], <>, the ``OffsetRange``s, <>, and `useConsumerCache` enabled). 159 | 160 | CAUTION: FIXME We all would appreciate if Jacek made the above less technical. 161 | 162 | CAUTION: FIXME What's `useConsumerCache`? 163 | 164 | With that, `compute` link:spark-streaming-InputInfoTracker.adoc#reportInfo[informs `InputInfoTracker` about the state of an input stream] (as link:spark-streaming-InputInfoTracker.adoc#StreamInputInfo[StreamInputInfo] with metadata with offsets and a human-friendly description). 165 | 166 | In the end, `compute` sets the just-calculated offsets as <>, <> (from <>) and returns the newly-created `KafkaRDD`. 167 | 168 | === [[commitAll]] Committing Queued Offsets to Kafka -- `commitAll` Method 169 | 170 | [source, scala] 171 | ---- 172 | commitAll(): Unit 173 | ---- 174 | 175 | `commitAll` commits all queued link:spark-streaming-kafka-HasOffsetRanges.adoc#OffsetRange[OffsetRange]s in <> (using Kafka's link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/Consumer.html#commitAsync(java.util.Map,%20org.apache.kafka.clients.consumer.OffsetCommitCallback)++[Consumer.commitAsync]). 176 | 177 | NOTE: `commitAll` is used for every batch interval (when <> is called to generate a `KafkaRDD`). 178 | 179 | Internally, `commitAll` walks through ``OffsetRange``s in <> and calculates the offsets for every topic partition. It uses them to create a collection of Kafka's https://kafka.apache.org/0100/javadoc/org/apache/kafka/common/TopicPartition.html[TopicPartition] and https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/OffsetAndMetadata.html[OffsetAndMetadata] pairs for Kafka's link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/Consumer.html#commitAsync(java.util.Map,%20org.apache.kafka.clients.consumer.OffsetCommitCallback)++[Consumer.commitAsync] using the internal <> reference. 180 | 181 | === [[clamp]] `clamp` Method 182 | 183 | [source, scala] 184 | ---- 185 | clamp(offsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] 186 | ---- 187 | 188 | `clamp` calls <> on the input `offsets` collection (of topic partitions with their offsets)... 189 | 190 | CAUTION: FIXME 191 | 192 | === [[maxMessagesPerPartition]] `maxMessagesPerPartition` Method 193 | 194 | CAUTION: FIXME 195 | 196 | === [[consumer]] Creating Kafka Consumer -- `consumer` Method 197 | 198 | [source, scala] 199 | ---- 200 | consumer(): Consumer[K, V] 201 | ---- 202 | 203 | `consumer` creates a Kafka `Consumer` with keys of type `K` and values of type `V` (specified when the <>). 204 | 205 | `consumer` starts the link:spark-streaming-kafka-ConsumerStrategy.adoc#onStart[ConsumerStrategy] (that was used when the `DirectKafkaInputDStream` was created). It passes the internal collection of ``TopicPartition``s and their offsets. 206 | 207 | CAUTION: FIXME A note with What `ConsumerStrategy` is for? 208 | 209 | === [[getPreferredHosts]] Calculating Preferred Hosts Using `LocationStrategy` -- `getPreferredHosts` Method 210 | 211 | [source, scala] 212 | ---- 213 | getPreferredHosts: java.util.Map[TopicPartition, String] 214 | ---- 215 | 216 | `getPreferredHosts` calculates preferred hosts per topic partition (that are later used to map link:spark-streaming-kafka-KafkaRDD.adoc[KafkaRDD] partitions to host leaders of topic partitions that Spark executors read records from). 217 | 218 | `getPreferredHosts` relies exclusively on the link:spark-streaming-kafka-LocationStrategy.adoc[LocationStrategy] that was passed in when <>. 219 | 220 | .DirectKafkaInputDStream.getPreferredHosts and Location Strategies 221 | [cols="1,2",options="header",width="100%"] 222 | |=== 223 | | Location Strategy | DirectKafkaInputDStream.getPreferredHosts 224 | | `PreferBrokers` 225 | | <>. 226 | 227 | | `PreferConsistent` 228 | | No host preference. Returns an empty collection of preferred hosts per topic partition. 229 | 230 | It does not call Kafka broker(s) for topic assignments. 231 | 232 | | `PreferFixed` 233 | | Returns the preferred hosts that were passed in when `PreferFixed` was created. 234 | 235 | It does not call Kafka broker(s) for topic assignments. 236 | |=== 237 | 238 | NOTE: `getPreferredHosts` is used when <>. 239 | 240 | ==== [[getBrokers]] Requesting Partition Assignments from Kafka -- `getBrokers` Method 241 | 242 | [source, scala] 243 | ---- 244 | getBrokers: ju.Map[TopicPartition, String] 245 | ---- 246 | 247 | `getBrokers` uses the internal <> instance to request Kafka broker(s) for partition assignments, i.e. the leader host per topic partition. 248 | 249 | NOTE: `getBrokers` uses Kafka's link:++https://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#assignment()++[Consumer.assignment()]. 250 | 251 | === [[stop]] Stopping DirectKafkaInputDStream -- `stop` Method 252 | 253 | [source, scala] 254 | ---- 255 | stop(): Unit 256 | ---- 257 | 258 | `stop` closes the internal <>. 259 | 260 | NOTE: `stop` is a part of the link:spark-streaming-inputdstreams.adoc[InputDStream Contract]. 261 | 262 | === [[latestOffsets]] Requesting Latest Offsets from Kafka Brokers -- `latestOffsets` Method 263 | 264 | [source, scala] 265 | ---- 266 | latestOffsets(): Map[TopicPartition, Long] 267 | ---- 268 | 269 | `latestOffsets` uses the internal <> to poll for the latest topic partition offsets, including partitions that have been added recently. 270 | 271 | `latestOffsets` calculates the topic partitions that are new (comparing to <>) and adds them to `currentOffsets`. 272 | 273 | NOTE: `latestOffsets` uses `poll(0)`, `assignment`, `position` (twice for every `TopicPartition`), `pause`, `seekToEnd` method calls. They _seem_ quite performance-heavy. Are they? 274 | 275 | The new partitions are ``pause``d and the current offsets ``seekToEnd``ed. 276 | 277 | CAUTION: FIXME Why are new partitions paused? Make the description more user-friendly. 278 | 279 | NOTE: `latestOffsets` is used when <>. 280 | 281 | === [[back-pressure]] Back Pressure 282 | 283 | CAUTION: FIXME 284 | 285 | link:spark-streaming-backpressure.adoc[Back pressure] for Direct Kafka input dstream can be configured using link:spark-streaming-settings.adoc#back-pressure[spark.streaming.backpressure.enabled] setting. 286 | 287 | NOTE: Back pressure is disabled by default. 288 | -------------------------------------------------------------------------------- /spark-streaming-streamingcontext.adoc: -------------------------------------------------------------------------------- 1 | == `StreamingContext` -- The Entry Point to Spark Streaming 2 | 3 | `StreamingContext` is the entry point for all Spark Streaming functionality. Whatever you do in Spark Streaming has to start from <>. 4 | 5 | [source, scala] 6 | ---- 7 | import org.apache.spark.streaming._ 8 | val sc = SparkContext.getOrCreate 9 | val ssc = new StreamingContext(sc, Seconds(5)) 10 | ---- 11 | 12 | NOTE: `StreamingContext` belongs to `org.apache.spark.streaming` package. 13 | 14 | With an instance of `StreamingContext` in your hands, you can <> or <>. 15 | 16 | Once streaming pipelines are developed, you <> to set the stream transformations in motion. You <> the instance when you are done. 17 | 18 | === [[creating-instance]] Creating Instance 19 | 20 | You can create a new instance of `StreamingContext` using the following constructors. You can group them by whether a StreamingContext constructor creates it from scratch or it is recreated from a checkpoint directory (follow the links for their extensive coverage). 21 | 22 | * <>: 23 | ** `StreamingContext(conf: SparkConf, batchDuration: Duration)` 24 | ** `StreamingContext(master: String, appName: String, batchDuration: Duration, sparkHome: String, jars: Seq[String], environment: Map[String,String])` 25 | ** `StreamingContext(sparkContext: SparkContext, batchDuration: Duration)` 26 | * link:spark-streaming-checkpointing.adoc#recreating-streamingcontext[Recreating StreamingContext from a checkpoint file] (where `path` is the <>): 27 | ** `StreamingContext(path: String)` 28 | ** `StreamingContext(path: String, hadoopConf: Configuration)` 29 | ** `StreamingContext(path: String, sparkContext: SparkContext)` 30 | 31 | NOTE: `StreamingContext(path: String)` uses link:../spark-SparkHadoopUtil.adoc#conf[SparkHadoopUtil.get.conf]. 32 | 33 | NOTE: When a StreamingContext is created and link:spark-streaming-settings.adoc#checkpointing[spark.streaming.checkpoint.directory] setting is set, the value gets passed on to <> method. 34 | 35 | ==== [[creating-instance-from-scratch]] Creating StreamingContext from Scratch 36 | 37 | When you create a new instance of `StreamingContext`, it first checks whether a link:spark-sparkcontext.adoc[SparkContext] or the <> are given (but not both!) 38 | 39 | [TIP] 40 | ==== 41 | `StreamingContext` will warn you when you use `local` or `local[1]` link:spark-deployment-environments.adoc#master-urls[master URLs]: 42 | 43 | [options="wrap"] 44 | ---- 45 | WARN StreamingContext: spark.master should be set as local[n], n > 1 in local mode if you have receivers to get data, otherwise Spark jobs will not get resources to process the received data. 46 | ---- 47 | ==== 48 | 49 | .StreamingContext and Dependencies 50 | image::images/spark-streaming-streamingcontext.png[align="center"] 51 | 52 | A link:spark-streaming-dstreamgraph.adoc[DStreamGraph] is created. 53 | 54 | A link:spark-streaming-jobscheduler.adoc[JobScheduler] is created. 55 | 56 | A link:spark-streaming-streaminglisteners.adoc#StreamingJobProgressListener[StreamingJobProgressListener] is created. 57 | 58 | link:spark-streaming-webui.adoc[Streaming tab] in web UI is created (when link:spark-webui.adoc#spark.ui.enabled[spark.ui.enabled] is enabled). 59 | 60 | A link:spark-streaming-StreamingSource.adoc[StreamingSource] is instantiated. 61 | 62 | At this point, `StreamingContext` enters <> state. 63 | 64 | === [[creating-receivers]] Creating ReceiverInputDStreams 65 | 66 | `StreamingContext` offers the following methods to create link:spark-streaming-receiverinputdstreams.adoc[ReceiverInputDStreams]: 67 | 68 | * <> 69 | * `actorStream[T](props: Props, name: String, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, supervisorStrategy: SupervisorStrategy = ActorSupervisorStrategy.defaultStrategy): ReceiverInputDStream[T]` 70 | * `socketTextStream(hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2): ReceiverInputDStream[String]` 71 | * `socketStream[T](hostname: String, port: Int, converter: (InputStream) => Iterator[T], storageLevel: StorageLevel): ReceiverInputDStream[T]` 72 | * `rawSocketStream[T](hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2): ReceiverInputDStream[T]` 73 | 74 | `StreamingContext` offers the following methods to create link:spark-streaming-inputdstreams.adoc[InputDStreams]: 75 | 76 | * `queueStream[T](queue: Queue[RDD[T]], oneAtATime: Boolean = true): InputDStream[T]` 77 | * `queueStream[T](queue: Queue[RDD[T]], oneAtATime: Boolean, defaultRDD: RDD[T]): InputDStream[T]` 78 | 79 | You can also use two additional methods in `StreamingContext` to build (or better called _compose_) a custom link:spark-streaming-dstreams.adoc[DStream]: 80 | 81 | * `union[T](streams: Seq[DStream[T]]): DStream[T]` 82 | * <> 83 | 84 | ==== [[receiverStream]] `receiverStream` method 85 | 86 | [source, scala] 87 | ---- 88 | receiverStream[T: ClassTag](receiver: Receiver[T]): ReceiverInputDStream[T] 89 | ---- 90 | 91 | You can register a custom input dstream using `receiverStream` method. It accepts a link:spark-streaming-receivers.adoc[Receiver]. 92 | 93 | NOTE: You can find an example of a custom `Receiver` in link:spark-streaming-receivers.adoc#custom-receiver[Custom Receiver]. 94 | 95 | === [[transform]] `transform` method 96 | 97 | [source, scala] 98 | ---- 99 | transform[T](dstreams: Seq[DStream[_]], transformFunc: (Seq[RDD[_]], Time) => RDD[T]): DStream[T] 100 | ---- 101 | 102 | ==== [[transform-example]] `transform` Example 103 | 104 | [source, scala] 105 | ---- 106 | import org.apache.spark.rdd.RDD 107 | def union(rdds: Seq[RDD[_]], time: Time) = { 108 | rdds.head.context.union(rdds.map(_.asInstanceOf[RDD[Int]])) 109 | } 110 | ssc.transform(Seq(cis), union) 111 | ---- 112 | 113 | === [[remember]] `remember` method 114 | 115 | [source, scala] 116 | ---- 117 | remember(duration: Duration): Unit 118 | ---- 119 | 120 | `remember` method sets the link:spark-streaming-dstreamgraph.adoc#remember-interval[remember interval] (for the graph of output dstreams). It simply calls link:spark-streaming-dstreamgraph.adoc#remember[DStreamGraph.remember] method and exits. 121 | 122 | CAUTION: FIXME figure 123 | 124 | === [[checkpoint-interval]][[checkpointDuration]] Checkpoint Interval 125 | 126 | The *checkpoint interval* is an internal property of `StreamingContext` and corresponds to link:spark-streaming-dstreamgraph.adoc#batch-interval[batch interval] or link:spark-streaming-checkpointing.adoc#Checkpoint[checkpoint interval of the checkpoint] (when <>). 127 | 128 | NOTE: The checkpoint interval property is also called *graph checkpointing interval*. 129 | 130 | <> when <> is defined (i.e. not `null`). 131 | 132 | === [[checkpointDir]][[checkpoint-directory]] Checkpoint Directory 133 | 134 | A *checkpoint directory* is a HDFS-compatible directory where link:spark-streaming-checkpointing.adoc[checkpoints] are written to. 135 | 136 | NOTE: _"A HDFS-compatible directory"_ means that it is Hadoop's Path class to handle all file system-related operations. 137 | 138 | Its initial value depends on whether the link:spark-streaming-checkpointing.adoc#recreating-streamingcontext[StreamingContext was (re)created from a checkpoint] or not, and is the checkpoint directory if so. Otherwise, it is not set (i.e. `null`). 139 | 140 | You can set the checkpoint directory when a <> or later using <> method. 141 | 142 | Internally, a checkpoint directory is tracked as `checkpointDir`. 143 | 144 | TIP: Refer to link:spark-streaming-checkpointing.adoc[Checkpointing] for more detailed coverage. 145 | 146 | === [[initial-checkpoint]][[initialCheckpoint]] Initial Checkpoint 147 | 148 | *Initial checkpoint* is the link:spark-streaming-checkpointing.adoc#recreating-streamingcontext[checkpoint (file) this StreamingContext has been recreated from]. 149 | 150 | The initial checkpoint is specified when a <>. 151 | 152 | [source, scala] 153 | ---- 154 | val ssc = new StreamingContext("_checkpoint") 155 | ---- 156 | 157 | === [[isCheckpointPresent]] Marking StreamingContext As Recreated from Checkpoint -- `isCheckpointPresent` method 158 | 159 | `isCheckpointPresent` internal method behaves like a flag that remembers whether the `StreamingContext` instance was created from a <> or not so the other internal parts of a streaming application can make decisions how to initialize themselves (or just be initialized). 160 | 161 | `isCheckpointPresent` checks the existence of the <> that gave birth to the StreamingContext. 162 | 163 | === [[checkpoint]] Setting Checkpoint Directory -- `checkpoint` method 164 | 165 | [source, scala] 166 | ---- 167 | checkpoint(directory: String): Unit 168 | ---- 169 | 170 | You use `checkpoint` method to set `directory` as the current <>. 171 | 172 | NOTE: Spark creates the directory unless it exists already. 173 | 174 | `checkpoint` uses link:spark-sparkcontext.adoc#hadoopConfiguration[SparkContext.hadoopConfiguration] to get the file system and create `directory` on. The full path of the directory is passed on to link:spark-sparkcontext.adoc#setCheckpointDir[SparkContext.setCheckpointDir] method. 175 | 176 | NOTE: Calling `checkpoint` with `null` as `directory` clears the checkpoint directory that effectively disables checkpointing. 177 | 178 | NOTE: When <> and link:spark-streaming-settings.adoc#checkpointing[spark.streaming.checkpoint.directory] setting is set, the value gets passed on to `checkpoint` method. 179 | 180 | === [[start]] Starting `StreamingContext` -- `start` method 181 | 182 | [source, scala] 183 | ---- 184 | start(): Unit 185 | ---- 186 | 187 | `start()` starts stream processing. It acts differently per <> and only <> state makes for a proper startup. 188 | 189 | NOTE: Consult <> section in this document to learn about the states of StreamingContext. 190 | 191 | ==== [[start-INITIALIZED]] Starting in INITIALIZED state 192 | 193 | Right after StreamingContext has been instantiated, it enters `INITIALIZED` state in which `start` first checks whether another `StreamingContext` instance has already been started in the JVM. It throws `IllegalStateException` exception if it was and exits. 194 | 195 | [options="wrap"] 196 | ---- 197 | java.lang.IllegalStateException: Only one StreamingContext may be started in this JVM. Currently running StreamingContext was started at [startSite] 198 | ---- 199 | 200 | If no other StreamingContext exists, it performs <> and link:spark-streaming-jobscheduler.adoc#start[starts `JobScheduler`] (in a separate dedicated daemon thread called *streaming-start*). 201 | 202 | .When started, StreamingContext starts JobScheduler 203 | image::images/spark-streaming-StreamingContext-start.png[align="center"] 204 | 205 | It enters <> state. 206 | 207 | It then register the <> and <>. If link:spark-webui.adoc#spark.ui.enabled[web UI is enabled], it attaches the link:spark-streaming-webui.adoc[Streaming tab]. 208 | 209 | Given all the above has have finished properly, it is assumed that the StreamingContext started fine and so you should see the following INFO message in the logs: 210 | 211 | ``` 212 | INFO StreamingContext: StreamingContext started 213 | ``` 214 | 215 | ==== [[start-ACTIVE]] Starting in ACTIVE state 216 | 217 | When in `ACTIVE` state, i.e. <>, executing `start` merely leads to the following WARN message in the logs: 218 | 219 | ``` 220 | WARN StreamingContext: StreamingContext has already been started 221 | ``` 222 | 223 | ==== [[start-STOPPED]] Starting in STOPPED state 224 | 225 | Attempting to start `StreamingContext` in <> state, i.e. <>, leads to the `IllegalStateException` exception: 226 | 227 | ``` 228 | java.lang.IllegalStateException: StreamingContext has already been stopped 229 | ``` 230 | 231 | === [[stop]][[stopping]] Stopping StreamingContext -- `stop` methods 232 | 233 | You stop `StreamingContext` using one of the three variants of `stop` method: 234 | 235 | * `stop(stopSparkContext: Boolean = true)` 236 | * `stop(stopSparkContext: Boolean, stopGracefully: Boolean)` 237 | 238 | NOTE: The first `stop` method uses link:spark-streaming-settings.adoc[spark.streaming.stopSparkContextByDefault] configuration setting that controls `stopSparkContext` input parameter. 239 | 240 | `stop` methods stop the execution of the streams immediately (`stopGracefully` is `false`) or wait for the processing of all received data to be completed (`stopGracefully` is `true`). 241 | 242 | `stop` reacts appropriately per the state of `StreamingContext`, but the end state is always <> state with shutdown hook removed. 243 | 244 | If a user requested to stop the underlying SparkContext (when `stopSparkContext` flag is enabled, i.e. `true`), link:spark-sparkcontext.adoc#stopping[it is now attempted to be stopped]. 245 | 246 | ==== [[stop-ACTIVE]] Stopping in ACTIVE state 247 | 248 | It is only in <> state when `stop` does more than printing out WARN messages to the logs. 249 | 250 | .StreamingContext Stop Procedure 251 | image::images/spark-streaming-StreamingContext-stop.png[align="center"] 252 | 253 | It does the following (in order): 254 | 255 | 1. link:spark-streaming-jobscheduler.adoc#stopping[JobScheduler is stopped]. 256 | 257 | 2. link:spark-streaming-StreamingSource.adoc[StreamingSource] is removed from link:../spark-MetricsSystem.adoc[MetricsSystem] (using `MetricsSystem.removeSource`) 258 | 259 | 3. link:spark-streaming-webui.adoc[Streaming tab] is detached (using `StreamingTab.detach`). 260 | 261 | 4. `ContextWaiter` is `notifyStop()` 262 | 263 | 5. `shutdownHookRef` is cleared. 264 | 265 | At that point, you should see the following INFO message in the logs: 266 | 267 | ``` 268 | INFO StreamingContext: StreamingContext stopped successfully 269 | ``` 270 | 271 | `StreamingContext` enters <> state. 272 | 273 | ==== [[stop-INITIALIZED]] Stopping in INITIALIZED state 274 | 275 | When in <> state, you should see the following WARN message in the logs: 276 | 277 | ``` 278 | WARN StreamingContext: StreamingContext has not been started yet 279 | ``` 280 | 281 | `StreamingContext` enters <> state. 282 | 283 | ==== [[stop-STOPPED]] Stopping in STOPPED state 284 | 285 | When in <> state, it prints the WARN message to the logs: 286 | 287 | ``` 288 | WARN StreamingContext: StreamingContext has already been stopped 289 | ``` 290 | 291 | `StreamingContext` enters <> state. 292 | 293 | === [[stopOnShutdown]] `stopOnShutdown` Shutdown Hook 294 | 295 | `stopOnShutdown` is a https://docs.oracle.com/javase/8/docs/api/java/lang/Runtime.html#addShutdownHook-java.lang.Thread-[JVM shutdown hook] to clean up after `StreamingContext` when the JVM shuts down, e.g. all non-daemon thread exited, `System.exit` was called or `^C` was typed. 296 | 297 | NOTE: It is registered to ShutdownHookManager when <>. 298 | 299 | NOTE: `ShutdownHookManager` uses `org.apache.hadoop.util.ShutdownHookManager` for its work. 300 | 301 | When executed, it first reads link:spark-streaming-settings.adoc[spark.streaming.stopGracefullyOnShutdown] setting that controls <>. You should see the following INFO message in the logs: 302 | 303 | ``` 304 | INFO Invoking stop(stopGracefully=[stopGracefully]) from shutdown hook 305 | ``` 306 | 307 | With the setting it <> without stopping the accompanying `SparkContext` (i.e. `stopSparkContext` parameter is disabled). 308 | 309 | === [[validate]] Setup Validation -- `validate` method 310 | 311 | [source, scala] 312 | ---- 313 | validate(): Unit 314 | ---- 315 | 316 | `validate()` method validates configuration of `StreamingContext`. 317 | 318 | NOTE: The method is executed when `StreamingContext` is <>. 319 | 320 | It first asserts that `DStreamGraph` has been assigned (i.e. `graph` field is not `null`) and triggers link:spark-streaming-dstreamgraph.adoc#dstreamgraph-validation[validation of DStreamGraph]. 321 | 322 | CAUTION: It appears that `graph` could never be `null`, though. 323 | 324 | If <>, it ensures that <> is set and checks whether the current streaming runtime environment can be safely serialized by link:spark-streaming-checkpointing.adoc#Checkpoint-serialize[serializing a checkpoint for fictitious batch time 0] (not link:spark-streaming-dstreamgraph.adoc#zero-time[zero time]). 325 | 326 | If link:spark-dynamic-allocation.adoc#isDynamicAllocationEnabled[dynamic allocation is enabled], it prints the following WARN message to the logs: 327 | 328 | [options="wrap"] 329 | ---- 330 | WARN StreamingContext: Dynamic Allocation is enabled for this application. Enabling Dynamic allocation for Spark Streaming applications can cause data loss if Write Ahead Log is not enabled for non-replayable sources like Flume. See the programming guide for details on how to enable the Write Ahead Log 331 | ---- 332 | 333 | === [[addStreamingListener]] Registering Streaming Listeners -- `addStreamingListener` method 334 | 335 | CAUTION: FIXME 336 | 337 | === [[streamingSource]] Streaming Metrics Source -- `streamingSource` Property 338 | 339 | CAUTION: FIXME 340 | 341 | === [[states]] States 342 | 343 | `StreamingContext` can be in three states: 344 | 345 | * `INITIALIZED`, i.e. after <>. 346 | * `ACTIVE`, i.e. after <>. 347 | * `STOPPED`, i.e. after <> 348 | --------------------------------------------------------------------------------