A very simple example of structured streaming from a Kafka source, where the messages
15 | are produced directly via calls to a KafkaProducer. A streaming DataFrame is created from a
16 | single Kafka topic, and feeds all the data received to a streaming computation that outputs it to a console.
17 |
Note that writing all the incremental data in each batch to output only makes sense because there is no
18 | aggregation performed. In subsequent examples with aggregation this will not be possible.
A streaming DataFrame is created from a single Kafka topic, an aggregating query is set up to count
25 | occurrences of each key, and the results are streamed to a console. Each batch results in the entire
26 | aggregation result to date being output.
Two Kafka topics are set up and a KafkaProducer is used to publish to the first topic.
33 | Then structured streaming is used to subscribe to that topic and publish a running aggregation to the
34 | second topic. Finally structured streaming is used to subscribe to the second topic and print the data received.
35 |
43 | The 'foreach' operation allows arbitrary computations on the output data in way that is both
44 | partition-aware (computed on the executors and aware of which partition is being processed) and batch-aware
45 | (via a separate invocation for each partition/batch combination.)
46 |
47 | It is always used by passing the operation an object that implements the 'ForeachWriter' interface. In this
48 | example, the object doesn't do any "useful" work: instead it is set up to illustrate its slightly arcane state
49 | management by printing its arguments and state in each of the three overridden methods.
50 |
51 | Each instance of ForeachWriter is used for processing a sequence of partition/batch combinations, but at any point
52 | in time is is setup (via a single open() call) to process one partition/batch combination. Then it gets multiple
53 | process() calls, providing the the actual data for that partition and batch, and then a single close() call to
54 | signal that the partition/batch combination has been completely processed.
55 |
56 |
57 |
58 |
59 |
60 |
61 |
62 |
--------------------------------------------------------------------------------
/src/main/scala/util/SimpleKafkaClient.scala:
--------------------------------------------------------------------------------
1 | package util
2 |
3 | import java.util
4 | import java.util.Properties
5 |
6 | import scala.collection.JavaConversions._
7 | import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecords, KafkaConsumer}
8 | import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
9 | import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer}
10 |
11 |
12 | /**
13 | * Simple utilities for connecting directly to Kafka.
14 | */
15 | class SimpleKafkaClient(server: EmbeddedKafkaServer) {
16 |
17 | def send(topic: String, pairs: Seq[(String, String)]) : Unit = {
18 | val producer = new KafkaProducer[String, String](basicStringStringProducer)
19 | pairs.foreach(pair => {
20 | producer send(new ProducerRecord(topic, pair._1, pair._2))
21 | })
22 | producer.close()
23 | }
24 |
25 | /**
26 | * Read and print the specified number of records from the specified topic.
27 | * Poll for as long as necessary.
28 | * @param topic
29 | * @param max
30 | */
31 | def consumeAndPrint(topic: String, max: Int): Unit = {
32 | // configure a consumer
33 |
34 |
35 | val consumer = new KafkaConsumer[String, String](basicStringStringConsumer);
36 |
37 | // need to subscribe to the topic
38 |
39 | consumer.subscribe(util.Arrays.asList(topic))
40 |
41 | // and read the records back -- just keep polling until we have read
42 | // all of them (poll each 100 msec) as the Kafka server may not make
43 | // them available immediately
44 |
45 | var count = 0;
46 |
47 | while (count < max) {
48 | println("*** Polling ")
49 |
50 | val records: ConsumerRecords[String, String] =
51 | consumer.poll(100)
52 | println(s"*** received ${records.count} messages")
53 | count = count + records.count
54 |
55 | // must specify the topic as we could have subscribed to more than one
56 | records.records(topic).foreach(rec => {
57 | println("*** [ " + rec.partition() + " ] " + rec.key() + ":" + rec.value())
58 | })
59 | }
60 |
61 | println("*** got the expected number of messages")
62 |
63 | consumer.close()
64 | }
65 |
66 | def basicStringStringProducer : Properties = {
67 | val config: Properties = new Properties
68 | config.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[StringSerializer].getCanonicalName)
69 | config.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[StringSerializer].getCanonicalName)
70 | config.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, server.getKafkaConnect)
71 | //config.put(ProducerConfig.PARTITIONER_CLASS_CONFIG, "org.apache.kafka.clients.producer.internals.DefaultPartitioner")
72 | config
73 | }
74 |
75 | def basicStringStringConsumer : Properties = {
76 | SimpleKafkaClient.getBasicStringStringConsumer(server)
77 | }
78 | }
79 |
80 | object SimpleKafkaClient {
81 |
82 | def getBasicStringStringConsumer(server: EmbeddedKafkaServer, group:String = "MyGroup") : Properties = {
83 | val consumerConfig: Properties = new Properties
84 | consumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, group)
85 | consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[StringDeserializer].getCanonicalName)
86 | consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[StringDeserializer].getCanonicalName)
87 | consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, server.getKafkaConnect)
88 | consumerConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
89 |
90 | //consumerConfig.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY, "roundrobin")
91 |
92 | consumerConfig
93 | }
94 |
95 | }
96 |
--------------------------------------------------------------------------------
/src/main/scala/structured/SubscribeAndPublish.scala:
--------------------------------------------------------------------------------
1 | package structured
2 |
3 | import java.io.File
4 |
5 | import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
6 | import org.apache.spark.sql.SparkSession
7 | import org.apache.spark.sql.streaming.{ProcessingTime, Trigger}
8 | import util.{TemporaryDirectories, EmbeddedKafkaServer, SimpleKafkaClient}
9 |
10 | /**
11 | * Two Kafka topics are set up and a KafkaProducer is used to publish to the first topic.
12 | * Then structured streaming is used to subscribe to that topic and publish a running aggregation to the
13 | * second topic. Finally structured streaming is used to subscribe to the second topic and print the data received.
14 | */
15 | object SubscribeAndPublish {
16 |
17 | def main (args: Array[String]) {
18 |
19 | val topic1 = "foo"
20 | val topic2 = "bar"
21 |
22 | println("*** starting Kafka server")
23 | val kafkaServer = new EmbeddedKafkaServer()
24 | kafkaServer.start()
25 | kafkaServer.createTopic(topic1, 4)
26 | kafkaServer.createTopic(topic2, 4)
27 |
28 | Thread.sleep(5000)
29 |
30 | // publish some messages
31 | println("*** Publishing messages")
32 | val max = 1000
33 | val client = new SimpleKafkaClient(kafkaServer)
34 | val numbers = 1 to max
35 | val producer = new KafkaProducer[String, String](client.basicStringStringProducer)
36 | numbers.foreach { n =>
37 | producer.send(new ProducerRecord(topic1, "key_" + n, "string_" + n))
38 | }
39 | Thread.sleep(5000)
40 |
41 | val checkpointPath = kafkaServer.tempDirs.checkpointPath
42 |
43 | println("*** Starting to stream")
44 |
45 | val spark = SparkSession
46 | .builder
47 | .appName("Structured_SubscribeAndPublish")
48 | .config("spark.master", "local[4]")
49 | .getOrCreate()
50 |
51 | import spark.implicits._
52 |
53 | val ds1 = spark
54 | .readStream
55 | .format("kafka")
56 | .option("kafka.bootstrap.servers", kafkaServer.getKafkaConnect)
57 | .option("subscribe", topic1)
58 | .option("startingOffsets", "earliest")
59 | .load()
60 |
61 | val counts = ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
62 | .groupBy()
63 | .count()
64 |
65 | val publishQuery =
66 | counts
67 | .selectExpr("'RunningCount' AS key", "CAST(count AS STRING) AS value")
68 | .writeStream
69 | .outputMode("complete")
70 | .format("kafka")
71 | .option("checkpointLocation", checkpointPath)
72 | .option("kafka.bootstrap.servers", kafkaServer.getKafkaConnect)
73 | .option("topic", topic2)
74 | .start()
75 |
76 | val ds2 = spark
77 | .readStream
78 | .format("kafka")
79 | .option("kafka.bootstrap.servers", kafkaServer.getKafkaConnect)
80 | .option("subscribe", topic2)
81 | .option("startingOffsets", "earliest")
82 | .load()
83 |
84 | val counts2 = ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
85 | .as[(String, String)]
86 |
87 | val query = counts2
88 | .writeStream
89 | .trigger(Trigger.ProcessingTime("4 seconds"))
90 | .format("console")
91 | .start()
92 |
93 | println("*** done setting up streaming")
94 |
95 | Thread.sleep(2000)
96 |
97 | println("*** publishing more messages")
98 | numbers.foreach { n =>
99 | producer.send(new ProducerRecord(topic1, "key_" + n, "string_" + n))
100 | }
101 |
102 | Thread.sleep(8000)
103 |
104 | println("*** Stopping stream")
105 | query.stop()
106 |
107 | query.awaitTermination()
108 | spark.stop()
109 |
110 | println("*** Streaming terminated")
111 |
112 | // stop Kafka
113 | println("*** Stopping Kafka")
114 | kafkaServer.stop()
115 |
116 | println("*** done")
117 | }
118 | }
--------------------------------------------------------------------------------
/src/main/scala/util/EmbeddedKafkaServer.scala:
--------------------------------------------------------------------------------
1 | package util
2 |
3 | import java.io.IOException
4 | import scala.collection.JavaConversions._
5 |
6 | import com.typesafe.scalalogging.Logger
7 |
8 | import kafka.admin.TopicCommand
9 | import kafka.server.{KafkaServerStartable, KafkaConfig}
10 | import kafka.utils.ZkUtils
11 |
12 | import org.apache.kafka.common.security.JaasUtils
13 |
14 | /**
15 | * A single embedded Kafka server and its associated Zookeeper
16 | */
17 | @throws[IOException]
18 | class EmbeddedKafkaServer() {
19 | private val LOGGER = Logger[EmbeddedKafkaServer]
20 | val tempDirs = new TemporaryDirectories
21 | val zkPort = 39001
22 | val kbPort = 39002
23 | val zkSessionTimeout = 20000
24 | val zkConnectionTimeout = 20000
25 |
26 | private var zookeeperHandle: Option[EmbeddedZookeeper] = None
27 | private var kafkaBrokerHandle: Option[KafkaServerStartable] = None
28 |
29 | /**
30 | * Start first the Zookeeper and then the Kafka broker.
31 | */
32 | def start() {
33 | LOGGER.info(s"starting on [$zkPort $kbPort]")
34 | zookeeperHandle = Some(new EmbeddedZookeeper(zkPort, tempDirs))
35 | zookeeperHandle.get.start
36 |
37 | val kafkaProps = Map(
38 | "port" -> Integer.toString(kbPort),
39 | "broker.id" -> "1",
40 | "host.name" -> "localhost",
41 | "log.dir" -> tempDirs.kafkaLogDirPath,
42 | "zookeeper.connect" -> ("localhost:" + zkPort))
43 |
44 | kafkaBrokerHandle = Some(new KafkaServerStartable(new KafkaConfig(kafkaProps)))
45 | kafkaBrokerHandle.get.startup()
46 | }
47 |
48 | /**
49 | * If running, shut down first the Kafka broker and then the Zookeeper
50 | */
51 | def stop() {
52 | LOGGER.info(s"shutting down broker on $kbPort")
53 | kafkaBrokerHandle match {
54 | case Some(b) => {
55 | b.shutdown()
56 | b.awaitShutdown()
57 | kafkaBrokerHandle = None
58 | }
59 | case None =>
60 | }
61 | Thread.sleep(5000)
62 | LOGGER.info(s"shutting down zookeeper on $zkPort")
63 | zookeeperHandle match {
64 | case Some(zk) => {
65 | zk.stop()
66 | zookeeperHandle = None
67 | }
68 | case None =>
69 | }
70 | }
71 |
72 | /**
73 | * Create a topic, optionally setting the number of partitions to a non default value and configuring timestamps.
74 | * @param topic
75 | * @param partitions
76 | * @param logAppendTime
77 | */
78 | def createTopic(topic: String, partitions: Int = 1, logAppendTime: Boolean = false) : Unit = {
79 | LOGGER.debug(s"Creating [$topic]")
80 |
81 | val arguments = Array[String](
82 | "--create",
83 | "--topic",
84 | topic
85 | ) ++ (
86 | if (logAppendTime) {
87 | Array[String]("--config", "message.timestamp.type=LogAppendTime")
88 | } else {
89 | Array[String]()
90 | }) ++ Array[String](
91 | "--partitions",
92 | "" + partitions,
93 | "--replication-factor",
94 | "1"
95 | )
96 |
97 | val opts = new TopicCommand.TopicCommandOptions(arguments)
98 |
99 | val zkUtils = ZkUtils.apply(getZkConnect,
100 | zkSessionTimeout, zkConnectionTimeout,
101 | JaasUtils.isZkSecurityEnabled)
102 |
103 | TopicCommand.createTopic(zkUtils, opts)
104 |
105 | LOGGER.debug(s"Finished creating topic [$topic]")
106 | }
107 |
108 | def addPartitions(topic: String, partitions: Int) : Unit = {
109 | LOGGER.debug(s"Adding [$partitions] partitions to [$topic]")
110 |
111 | val arguments = Array[String](
112 | "--alter",
113 | "--topic",
114 | topic,
115 | "--partitions",
116 | "" + partitions
117 | )
118 |
119 | val opts = new TopicCommand.TopicCommandOptions(arguments)
120 |
121 | val zkUtils = ZkUtils.apply(getZkConnect,
122 | zkSessionTimeout, zkConnectionTimeout,
123 | JaasUtils.isZkSecurityEnabled)
124 |
125 | TopicCommand.alterTopic(zkUtils, opts)
126 |
127 | LOGGER.debug(s"Finished adding [$partitions] partitions to [$topic]")
128 | }
129 |
130 | def getKafkaConnect: String = "localhost:" + kbPort
131 |
132 | def getZkConnect: String = "localhost:" + zkPort
133 |
134 |
135 | }
136 |
--------------------------------------------------------------------------------
/src/main/scala/ControlledPartitioning.scala:
--------------------------------------------------------------------------------
1 | import java.util.{Arrays, Properties}
2 |
3 | import kafka.serializer.StringDecoder
4 | import org.apache.spark.streaming.kafka010.{ConsumerStrategies, KafkaUtils, LocationStrategies}
5 | import org.apache.spark.streaming.{Seconds, StreamingContext}
6 | import org.apache.spark.{SparkConf, SparkContext}
7 | import util.{EmbeddedKafkaServer, SimpleKafkaClient, SparkKafkaSink}
8 |
9 | /**
10 | * Here the topic has six partitions but instead of writing to it using the configured
11 | * partitioner, we assign all records to the same partition explicitly. Although the
12 | * generated RDDs still have the same number of partitions as the topic, only one
13 | * partition has all the data in it. THis is a rather extreme way to use topic partitions,
14 | * but it opens up the whole range of algorithms for selecting the partition when sending.
15 | */
16 | object ControlledPartitioning {
17 |
18 | /**
19 | * Publish some data to a topic. Encapsulated here to ensure serializability.
20 | * @param max
21 | * @param sc
22 | * @param topic
23 | * @param config
24 | */
25 | def send(max: Int, sc: SparkContext, topic: String, config: Properties): Unit = {
26 |
27 | // put some data in an RDD and publish to Kafka
28 | val numbers = 1 to max
29 | val numbersRDD = sc.parallelize(numbers, 5)
30 |
31 | val kafkaSink = sc.broadcast(SparkKafkaSink(config))
32 |
33 | println("*** producing data")
34 |
35 | // use the overload that explicitly assigns a partition (0)
36 | numbersRDD.foreach { n =>
37 | kafkaSink.value.send(topic, 0, "key_" + n, "string_" + n)
38 | }
39 | }
40 |
41 | def main (args: Array[String]) {
42 |
43 | val topic = "foo"
44 |
45 | val kafkaServer = new EmbeddedKafkaServer()
46 | kafkaServer.start()
47 | kafkaServer.createTopic(topic, 6)
48 |
49 |
50 |
51 | val conf = new SparkConf().setAppName("ControlledPartitioning").setMaster("local[7]")
52 | val sc = new SparkContext(conf)
53 |
54 | // streams will produce data every second
55 | val ssc = new StreamingContext(sc, Seconds(1))
56 |
57 | val max = 1000
58 |
59 | val props: Properties = SimpleKafkaClient.getBasicStringStringConsumer(kafkaServer)
60 |
61 | val kafkaStream =
62 | KafkaUtils.createDirectStream(
63 | ssc,
64 | LocationStrategies.PreferConsistent,
65 | ConsumerStrategies.Subscribe[String, String](
66 | Arrays.asList(topic),
67 | props.asInstanceOf[java.util.Map[String, Object]]
68 | )
69 |
70 | )
71 |
72 | // now, whenever this Kafka stream produces data the resulting RDD will be printed
73 | kafkaStream.foreachRDD(r => {
74 | println("*** got an RDD, size = " + r.count())
75 | r.foreach(s => println(s))
76 | if (r.count() > 0) {
77 | // let's see how many partitions the resulting RDD has -- notice that it has nothing
78 | // to do with the number of partitions in the RDD used to publish the data (4), nor
79 | // the number of partitions of the topic (which also happens to be four.)
80 | println("*** " + r.getNumPartitions + " partitions")
81 | r.glom().foreach(a => println("*** partition size = " + a.size))
82 | }
83 | })
84 |
85 | ssc.start()
86 |
87 | println("*** started termination monitor")
88 |
89 | // streams seem to need some time to get going
90 | Thread.sleep(5000)
91 |
92 | val producerThread = new Thread("Streaming Termination Controller") {
93 | override def run() {
94 | val client = new SimpleKafkaClient(kafkaServer)
95 |
96 | send(max, sc, topic, client.basicStringStringProducer)
97 | Thread.sleep(5000)
98 | println("*** requesting streaming termination")
99 | ssc.stop(stopSparkContext = false, stopGracefully = true)
100 | }
101 | }
102 | producerThread.start()
103 |
104 | try {
105 | ssc.awaitTermination()
106 | println("*** streaming terminated")
107 | } catch {
108 | case e: Exception => {
109 | println("*** streaming exception caught in monitor thread")
110 | }
111 | }
112 |
113 | // stop Spark
114 | sc.stop()
115 |
116 | // stop Kafka
117 | kafkaServer.stop()
118 |
119 | println("*** done")
120 | }
121 | }
--------------------------------------------------------------------------------
/src/main/scala/SendWithDifferentPartitioning.scala:
--------------------------------------------------------------------------------
1 | import java.util.{Arrays, Properties}
2 |
3 | import kafka.serializer.StringDecoder
4 | import org.apache.spark.streaming.kafka010.{ConsumerStrategies, KafkaUtils, LocationStrategies}
5 | import org.apache.spark.streaming.{Seconds, StreamingContext}
6 | import org.apache.spark.{SparkConf, SparkContext}
7 | import util.{EmbeddedKafkaServer, SimpleKafkaClient, SparkKafkaSink}
8 |
9 | /**
10 | * This example is very similar to SimpleStreaming, except that the data is sent
11 | * from an RDD with 5 partitions to a Kafka topic with 6 partitions. WThe KafkaStream consuming
12 | * the topic produces RDDs with size partitions. This is because the data is repartitioned when sent,
13 | * as we continue use the KafkaProducer constructor overload that doesn't allow us to specify
14 | * the destination partition.
15 | */
16 | object SendWithDifferentPartitioning {
17 |
18 | /**
19 | * Publish some data to a topic. Encapsulated here to ensure serializability.
20 | * @param max
21 | * @param sc
22 | * @param topic
23 | * @param config
24 | */
25 | def send(max: Int, sc: SparkContext, topic: String, config: Properties): Unit = {
26 |
27 | // put some data in an RDD and publish to Kafka
28 | val numbers = 1 to max
29 | val numbersRDD = sc.parallelize(numbers, 5)
30 |
31 | val kafkaSink = sc.broadcast(SparkKafkaSink(config))
32 |
33 | println("*** producing data")
34 |
35 | numbersRDD.foreach { n =>
36 | // NOTE:
37 | // 1) the keys and values are strings, which is important when receiving them
38 | // 2) We don't specify which Kafka partition to send to, so a hash of the key
39 | // is used to determine this
40 | kafkaSink.value.send(topic, "key_" + n, "string_" + n)
41 | }
42 | }
43 |
44 | def main (args: Array[String]) {
45 |
46 | val topic = "foo"
47 |
48 | val kafkaServer = new EmbeddedKafkaServer()
49 | kafkaServer.start()
50 | kafkaServer.createTopic(topic, 6)
51 |
52 |
53 |
54 | val conf = new SparkConf().setAppName("SendWithDifferentPartitioning").setMaster("local[7]")
55 | val sc = new SparkContext(conf)
56 |
57 | // streams will produce data every second
58 | val ssc = new StreamingContext(sc, Seconds(1))
59 |
60 | val max = 1000
61 |
62 | val props: Properties = SimpleKafkaClient.getBasicStringStringConsumer(kafkaServer)
63 |
64 | val kafkaStream =
65 | KafkaUtils.createDirectStream(
66 | ssc,
67 | LocationStrategies.PreferConsistent,
68 | ConsumerStrategies.Subscribe[String, String](
69 | Arrays.asList(topic),
70 | props.asInstanceOf[java.util.Map[String, Object]]
71 | )
72 |
73 | )
74 |
75 | // now, whenever this Kafka stream produces data the resulting RDD will be printed
76 | kafkaStream.foreachRDD(r => {
77 | println("*** got an RDD, size = " + r.count())
78 | r.foreach(s => println(s))
79 | if (r.count() > 0) {
80 | // let's see how many partitions the resulting RDD has -- notice that it has nothing
81 | // to do with the number of partitions in the RDD used to publish the data (4), nor
82 | // the number of partitions of the topic (which also happens to be four.)
83 | println("*** " + r.getNumPartitions + " partitions")
84 | r.glom().foreach(a => println("*** partition size = " + a.size))
85 | }
86 | })
87 |
88 | ssc.start()
89 |
90 | println("*** started termination monitor")
91 |
92 | // streams seem to need some time to get going
93 | Thread.sleep(5000)
94 |
95 | val producerThread = new Thread("Streaming Termination Controller") {
96 | override def run() {
97 | val client = new SimpleKafkaClient(kafkaServer)
98 |
99 | send(max, sc, topic, client.basicStringStringProducer)
100 | Thread.sleep(5000)
101 | println("*** requesting streaming termination")
102 | ssc.stop(stopSparkContext = false, stopGracefully = true)
103 | }
104 | }
105 | producerThread.start()
106 |
107 | try {
108 | ssc.awaitTermination()
109 | println("*** streaming terminated")
110 | } catch {
111 | case e: Exception => {
112 | println("*** streaming exception caught in monitor thread")
113 | }
114 | }
115 |
116 | // stop Spark
117 | sc.stop()
118 |
119 | // stop Kafka
120 | kafkaServer.stop()
121 |
122 | println("*** done")
123 | }
124 | }
--------------------------------------------------------------------------------
/src/main/scala/SimpleStreaming.scala:
--------------------------------------------------------------------------------
1 | import java.util.Properties
2 | import java.util.Arrays
3 |
4 | import org.apache.spark.streaming.{Seconds, StreamingContext}
5 | import org.apache.kafka.common.serialization.StringDeserializer
6 | import org.apache.spark.{SparkConf, SparkContext}
7 | import util.{EmbeddedKafkaServer, SimpleKafkaClient, SparkKafkaSink}
8 | import java.util
9 |
10 | import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
11 | import org.apache.spark.broadcast.Broadcast
12 | import org.apache.spark.streaming.kafka010.{ConsumerStrategies, KafkaUtils, LocationStrategies}
13 |
14 | /**
15 | * The most basic streaming example: starts a Kafka server, creates a topic, creates a stream
16 | * to process that topic, and publishes some data using the SparkKafkaSink.
17 | *
18 | * Notice there's quite a lot of waiting. It takes some time for streaming to get going,
19 | * and data published too early tends to be missed by the stream. (No doubt, this is partly
20 | * because this example uses the simplest method to create the stream, and thus doesn't
21 | * get an opportunity to set auto.offset.reset to "earliest".
22 | *
23 | * Also, data that is published takes some time to propagate to the stream.
24 | * This seems inevitable, and is almost guaranteed to be slower
25 | * in a self-contained example like this.
26 | */
27 | object SimpleStreaming {
28 |
29 | def main (args: Array[String]) {
30 |
31 | val topic = "foo"
32 |
33 | val kafkaServer = new EmbeddedKafkaServer()
34 | kafkaServer.start()
35 | kafkaServer.createTopic(topic, 4)
36 |
37 |
38 |
39 | val conf = new SparkConf().setAppName("SimpleStreaming").setMaster("local[4]")
40 | val sc = new SparkContext(conf)
41 |
42 | // streams will produce data every second
43 | val ssc = new StreamingContext(sc, Seconds(1))
44 |
45 | // this many messages
46 | val max = 1000
47 |
48 | // Create the stream.
49 | val props: Properties = SimpleKafkaClient.getBasicStringStringConsumer(kafkaServer)
50 |
51 | val kafkaStream =
52 | KafkaUtils.createDirectStream(
53 | ssc,
54 | LocationStrategies.PreferConsistent,
55 | ConsumerStrategies.Subscribe[String, String](
56 | Arrays.asList(topic),
57 | props.asInstanceOf[java.util.Map[String, Object]]
58 | )
59 |
60 | )
61 |
62 | // now, whenever this Kafka stream produces data the resulting RDD will be printed
63 | kafkaStream.foreachRDD(r => {
64 | println("*** got an RDD, size = " + r.count())
65 | r.foreach(s => println(s))
66 | if (r.count() > 0) {
67 | // let's see how many partitions the resulting RDD has -- notice that it has nothing
68 | // to do with the number of partitions in the RDD used to publish the data (4), nor
69 | // the number of partitions of the topic (which also happens to be four.)
70 | println("*** " + r.getNumPartitions + " partitions")
71 | r.glom().foreach(a => println("*** partition size = " + a.size))
72 | }
73 | })
74 |
75 | ssc.start()
76 |
77 | println("*** started termination monitor")
78 |
79 | // streams seem to need some time to get going
80 | Thread.sleep(5000)
81 |
82 | val producerThread = new Thread("Streaming Termination Controller") {
83 | override def run() {
84 | val client = new SimpleKafkaClient(kafkaServer)
85 |
86 | val numbers = 1 to max
87 |
88 | val producer = new KafkaProducer[String, String](client.basicStringStringProducer)
89 |
90 | numbers.foreach { n =>
91 | // NOTE:
92 | // 1) the keys and values are strings, which is important when receiving them
93 | // 2) We don't specify which Kafka partition to send to, so a hash of the key
94 | // is used to determine this
95 | producer.send(new ProducerRecord(topic, "key_" + n, "string_" + n))
96 | }
97 | Thread.sleep(5000)
98 | println("*** requesting streaming termination")
99 | ssc.stop(stopSparkContext = false, stopGracefully = true)
100 | }
101 | }
102 | producerThread.start()
103 |
104 | try {
105 | ssc.awaitTermination()
106 | println("*** streaming terminated")
107 | } catch {
108 | case e: Exception => {
109 | println("*** streaming exception caught in monitor thread")
110 | }
111 | }
112 |
113 | // stop Spark
114 | sc.stop()
115 |
116 | // stop Kafka
117 | kafkaServer.stop()
118 |
119 | println("*** done")
120 | }
121 | }
--------------------------------------------------------------------------------
/src/main/scala/MultipleTopics.scala:
--------------------------------------------------------------------------------
1 | import java.util.{Arrays, Properties}
2 |
3 | import org.apache.kafka.clients.consumer.ConsumerRecord
4 | import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
5 | import org.apache.spark.streaming.kafka010.{ConsumerStrategies, KafkaUtils, LocationStrategies}
6 | import org.apache.spark.streaming.{Seconds, StreamingContext}
7 | import org.apache.spark.{SparkConf, SparkContext}
8 | import util.{EmbeddedKafkaServer, PartitionMapAnalyzer, SimpleKafkaClient}
9 |
10 | /**
11 | * A single stream subscribing to the two topics receives data from both of them.
12 | * The partitioning behavior here is quite interesting, as the topics have three and six partitions respectively,
13 | * each RDD has nine partitions, and each RDD partition receives data from exactly one partition of one topic.
14 | *
15 | * Partitioning is analyzed using the PartitionMapAnalyzer.
16 | */
17 | object MultipleTopics {
18 |
19 | def main (args: Array[String]) {
20 |
21 | val topic1 = "foo"
22 | val topic2 = "bar"
23 |
24 | // topics are partitioned differently
25 | val kafkaServer = new EmbeddedKafkaServer()
26 | kafkaServer.start()
27 | kafkaServer.createTopic(topic1, 3)
28 | kafkaServer.createTopic(topic2, 6)
29 |
30 | val conf = new SparkConf().setAppName("MultipleTopics").setMaster("local[10]")
31 | val sc = new SparkContext(conf)
32 |
33 | // streams will produce data every second
34 | val ssc = new StreamingContext(sc, Seconds(1))
35 |
36 | // this many messages
37 | val max = 100
38 |
39 | // Create the stream.
40 | val props: Properties = SimpleKafkaClient.getBasicStringStringConsumer(kafkaServer)
41 |
42 | val kafkaStream =
43 | KafkaUtils.createDirectStream(
44 | ssc,
45 | LocationStrategies.PreferConsistent,
46 | ConsumerStrategies.Subscribe[String, String](
47 | Arrays.asList(topic1, topic2),
48 | props.asInstanceOf[java.util.Map[String, Object]]
49 | )
50 |
51 | )
52 |
53 | // now, whenever this Kafka stream produces data the resulting RDD will be printed
54 | kafkaStream.foreachRDD(r => {
55 | println("*** got an RDD, size = " + r.count())
56 |
57 | PartitionMapAnalyzer.analyze(r)
58 |
59 | })
60 |
61 | ssc.start()
62 |
63 | println("*** started streaming context")
64 |
65 | // streams seem to need some time to get going
66 | Thread.sleep(5000)
67 |
68 | val producerThreadTopic1 = new Thread("Producer thread 1") {
69 | override def run() {
70 | val client = new SimpleKafkaClient(kafkaServer)
71 |
72 | val numbers = 1 to max
73 |
74 | val producer = new KafkaProducer[String, String](client.basicStringStringProducer)
75 |
76 | numbers.foreach { n =>
77 | // NOTE:
78 | // 1) the keys and values are strings, which is important when receiving them
79 | // 2) We don't specify which Kafka partition to send to, so a hash of the key
80 | // is used to determine this
81 | producer.send(new ProducerRecord(topic1, "key_1_" + n, "string_1_" + n))
82 | }
83 |
84 | }
85 | }
86 |
87 | val producerThreadTopic2 = new Thread("Producer thread 2; controlling termination") {
88 | override def run() {
89 | val client = new SimpleKafkaClient(kafkaServer)
90 |
91 | val numbers = 1 to max
92 |
93 | val producer = new KafkaProducer[String, String](client.basicStringStringProducer)
94 |
95 | numbers.foreach { n =>
96 | // NOTE:
97 | // 1) the keys and values are strings, which is important when receiving them
98 | // 2) We don't specify which Kafka partition to send to, so a hash of the key
99 | // is used to determine this
100 | producer.send(new ProducerRecord(topic2, "key_2_" + n, "string_2_" + n))
101 | }
102 | Thread.sleep(10000)
103 | println("*** requesting streaming termination")
104 | ssc.stop(stopSparkContext = false, stopGracefully = true)
105 | }
106 | }
107 |
108 | producerThreadTopic1.start()
109 | producerThreadTopic2.start()
110 |
111 | try {
112 | ssc.awaitTermination()
113 | println("*** streaming terminated")
114 | } catch {
115 | case e: Exception => {
116 | println("*** streaming exception caught in monitor thread")
117 | }
118 | }
119 |
120 | // stop Spark
121 | sc.stop()
122 |
123 | // stop Kafka
124 | kafkaServer.stop()
125 |
126 | println("*** done")
127 | }
128 | }
--------------------------------------------------------------------------------
/src/main/scala/MultipleStreams.scala:
--------------------------------------------------------------------------------
1 | import java.util.{Arrays, Properties}
2 |
3 | import org.apache.spark.streaming.kafka010.{ConsumerStrategies, KafkaUtils, LocationStrategies}
4 | import org.apache.spark.streaming.{Seconds, StreamingContext}
5 | import org.apache.spark.{SparkConf, SparkContext}
6 | import util.{EmbeddedKafkaServer, SimpleKafkaClient, SparkKafkaSink}
7 |
8 | /**
9 | * This example creates two streams based on a single consumer group, so they divide up the data.
10 | * There's an interesting partitioning interaction here as the streams each get data from two fo the four
11 | * topic partitions, and each produce RDDs with two partitions each.
12 | */
13 |
14 | object MultipleStreams {
15 |
16 | /**
17 | * Publish some data to a topic. Encapsulated here to ensure serializability.
18 | * @param max
19 | * @param sc
20 | * @param topic
21 | * @param config
22 | */
23 | def send(max: Int, sc: SparkContext, topic: String, config: Properties): Unit = {
24 |
25 | // put some data in an RDD and publish to Kafka
26 | val numbers = 1 to max
27 | val numbersRDD = sc.parallelize(numbers, 4)
28 |
29 | val kafkaSink = sc.broadcast(SparkKafkaSink(config))
30 |
31 | println("*** producing data")
32 |
33 | numbersRDD.foreach { n =>
34 | kafkaSink.value.send(topic, "key_" + n, "string_" + n)
35 | }
36 | }
37 |
38 | def main (args: Array[String]) {
39 |
40 | val topic = "foo"
41 |
42 | val kafkaServer = new EmbeddedKafkaServer()
43 | kafkaServer.start()
44 | kafkaServer.createTopic(topic, 4)
45 |
46 | val conf = new SparkConf().setAppName("MultipleStreams").setMaster("local[4]")
47 | val sc = new SparkContext(conf)
48 |
49 | // streams will produce data every second
50 | val ssc = new StreamingContext(sc, Seconds(1))
51 |
52 | val max = 1000
53 |
54 |
55 | //
56 | // the first stream subscribes to the default consumer group in our SParkKafkaClient class
57 | //
58 |
59 | val props: Properties = SimpleKafkaClient.getBasicStringStringConsumer(kafkaServer)
60 |
61 | val kafkaStream1 =
62 | KafkaUtils.createDirectStream(
63 | ssc,
64 | LocationStrategies.PreferConsistent,
65 | ConsumerStrategies.Subscribe[String, String](
66 | Arrays.asList(topic),
67 | props.asInstanceOf[java.util.Map[String, Object]]
68 | )
69 |
70 | )
71 |
72 | kafkaStream1.foreachRDD(r => {
73 | println("*** [stream 1] got an RDD, size = " + r.count())
74 | r.foreach(s => println("*** [stream 1] " + s))
75 | if (r.count() > 0) {
76 | println("*** [stream 1] " + r.getNumPartitions + " partitions")
77 | r.glom().foreach(a => println("*** [stream 1] partition size = " + a.size))
78 | }
79 | })
80 |
81 | //
82 | // a second stream, uses the same props and hence the same consumer group
83 | //
84 |
85 | val kafkaStream2 =
86 | KafkaUtils.createDirectStream(
87 | ssc,
88 | LocationStrategies.PreferConsistent,
89 | ConsumerStrategies.Subscribe[String, String](
90 | Arrays.asList(topic),
91 | props.asInstanceOf[java.util.Map[String, Object]]
92 | )
93 |
94 | )
95 |
96 | kafkaStream2.foreachRDD(r => {
97 | println("*** [stream 2] got an RDD, size = " + r.count())
98 | r.foreach(s => println("*** [stream 2] " + s))
99 | if (r.count() > 0) {
100 | println("*** [stream 2] " + r.getNumPartitions + " partitions")
101 | r.glom().foreach(a => println("*** [stream 2] partition size = " + a.size))
102 | }
103 | })
104 |
105 | ssc.start()
106 |
107 | println("*** started termination monitor")
108 |
109 | // streams seem to need some time to get going
110 | Thread.sleep(5000)
111 |
112 | val producerThread = new Thread("Streaming Termination Controller") {
113 | override def run() {
114 | val client = new SimpleKafkaClient(kafkaServer)
115 |
116 | send(max, sc, topic, client.basicStringStringProducer)
117 | Thread.sleep(5000)
118 | println("*** requesting streaming termination")
119 | ssc.stop(stopSparkContext = false, stopGracefully = true)
120 | }
121 | }
122 | producerThread.start()
123 |
124 | try {
125 | ssc.awaitTermination()
126 | println("*** streaming terminated")
127 | } catch {
128 | case e: Exception => {
129 | println("*** streaming exception caught in monitor thread")
130 | }
131 | }
132 |
133 | // stop Spark
134 | sc.stop()
135 |
136 | // stop Kafka
137 | kafkaServer.stop()
138 |
139 | println("*** done")
140 | }
141 | }
--------------------------------------------------------------------------------
/src/main/scala/SimpleStreamingFromRDD.scala:
--------------------------------------------------------------------------------
1 | import java.util.{Arrays, Properties}
2 |
3 | import org.apache.spark.streaming.kafka010.{ConsumerStrategies, KafkaUtils, LocationStrategies}
4 | import org.apache.spark.streaming.{Seconds, StreamingContext}
5 | import org.apache.spark.{SparkConf, SparkContext}
6 | import util.{EmbeddedKafkaServer, SimpleKafkaClient, SparkKafkaSink}
7 |
8 | /**
9 | * The most basic streaming example: starts a Kafka server, creates a topic, creates a stream
10 | * to process that topic, and publishes some data using the SparkKafkaSink.
11 | *
12 | * Notice there's quite a lot of waiting. It takes some time for streaming to get going,
13 | * and data published too early tends to be missed by the stream. (No doubt, this is partly
14 | * because this example uses the simplest method to create the stream, and thus doesn't
15 | * get an opportunity to set auto.offset.reset to "earliest".
16 | *
17 | * Also, data that is published takes some time to propagate to the stream.
18 | * This seems inevitable, and is almost guaranteed to be slower
19 | * in a self-contained example like this.
20 | */
21 | object SimpleStreamingFromRDD {
22 |
23 | /**
24 | * Publish some data to a topic. Encapsulated here to ensure serializability.
25 | * @param max
26 | * @param sc
27 | * @param topic
28 | * @param config
29 | */
30 | def send(max: Int, sc: SparkContext, topic: String, config: Properties): Unit = {
31 |
32 | // put some data in an RDD and publish to Kafka
33 | val numbers = 1 to max
34 | val numbersRDD = sc.parallelize(numbers, 4)
35 |
36 | val kafkaSink = sc.broadcast(SparkKafkaSink(config))
37 |
38 | println("*** producing data")
39 |
40 | numbersRDD.foreach { n =>
41 | // NOTE:
42 | // 1) the keys and values are strings, which is important when receiving them
43 | // 2) We don't specify which Kafka partition to send to, so a hash of the key
44 | // is used to determine this
45 | kafkaSink.value.send(topic, "key_" + n, "string_" + n)
46 | }
47 | }
48 |
49 | def main (args: Array[String]) {
50 |
51 | val topic = "foo"
52 |
53 | val kafkaServer = new EmbeddedKafkaServer()
54 | kafkaServer.start()
55 | kafkaServer.createTopic(topic, 4)
56 |
57 |
58 |
59 | val conf = new SparkConf().setAppName("SimpleStreamingFromRDD").setMaster("local[4]")
60 | val sc = new SparkContext(conf)
61 |
62 | // streams will produce data every second
63 | val ssc = new StreamingContext(sc, Seconds(1))
64 |
65 | // this many messages
66 | val max = 1000
67 |
68 | // Create the stream.
69 | val props: Properties = SimpleKafkaClient.getBasicStringStringConsumer(kafkaServer)
70 |
71 | val kafkaStream =
72 | KafkaUtils.createDirectStream(
73 | ssc,
74 | LocationStrategies.PreferConsistent,
75 | ConsumerStrategies.Subscribe[String, String](
76 | Arrays.asList(topic),
77 | props.asInstanceOf[java.util.Map[String, Object]]
78 | )
79 |
80 | )
81 |
82 | // now, whenever this Kafka stream produces data the resulting RDD will be printed
83 | kafkaStream.foreachRDD(r => {
84 | println("*** got an RDD, size = " + r.count())
85 | r.foreach(s => println(s))
86 | if (r.count() > 0) {
87 | // let's see how many partitions the resulting RDD has -- notice that it has nothing
88 | // to do with the number of partitions in the RDD used to publish the data (4), nor
89 | // the number of partitions of the topic (which also happens to be four.)
90 | println("*** " + r.getNumPartitions + " partitions")
91 | r.glom().foreach(a => println("*** partition size = " + a.size))
92 | }
93 | })
94 |
95 | ssc.start()
96 |
97 | println("*** started termination monitor")
98 |
99 | // streams seem to need some time to get going
100 | Thread.sleep(5000)
101 |
102 | val producerThread = new Thread("Streaming Termination Controller") {
103 | override def run() {
104 | val client = new SimpleKafkaClient(kafkaServer)
105 |
106 | send(max, sc, topic, client.basicStringStringProducer)
107 |
108 | Thread.sleep(5000)
109 | println("*** requesting streaming termination")
110 | ssc.stop(stopSparkContext = false, stopGracefully = true)
111 | }
112 | }
113 | producerThread.start()
114 |
115 | try {
116 | ssc.awaitTermination()
117 | println("*** streaming terminated")
118 | } catch {
119 | case e: Exception => {
120 | println("*** streaming exception caught in monitor thread")
121 | }
122 | }
123 |
124 | // stop Spark
125 | sc.stop()
126 |
127 | // stop Kafka
128 | kafkaServer.stop()
129 |
130 | println("*** done")
131 | }
132 | }
--------------------------------------------------------------------------------
/src/main/scala/MultipleConsumerGroups.scala:
--------------------------------------------------------------------------------
1 | import java.util.{Arrays, Properties}
2 |
3 | import org.apache.spark.streaming.kafka010.{ConsumerStrategies, KafkaUtils, LocationStrategies}
4 | import org.apache.spark.streaming.{Seconds, StreamingContext}
5 | import org.apache.spark.{SparkConf, SparkContext}
6 | import util.{EmbeddedKafkaServer, SimpleKafkaClient, SparkKafkaSink}
7 |
8 | /**
9 | * This example creates two streams based on two different consumer groups, so both streams
10 | * get a copy of the same data. It's simply a matter of specifying the two names of the
11 | * two different consumer groups in the two calls to createStream() -- no special
12 | * configuration is needed.
13 | */
14 |
15 | object MultipleConsumerGroups {
16 |
17 | /**
18 | * Publish some data to a topic. Encapsulated here to ensure serializability.
19 | * @param max
20 | * @param sc
21 | * @param topic
22 | * @param config
23 | */
24 | def send(max: Int, sc: SparkContext, topic: String, config: Properties): Unit = {
25 |
26 | // put some data in an RDD and publish to Kafka
27 | val numbers = 1 to max
28 | val numbersRDD = sc.parallelize(numbers, 4)
29 |
30 | val kafkaSink = sc.broadcast(SparkKafkaSink(config))
31 |
32 | println("*** producing data")
33 |
34 | numbersRDD.foreach { n =>
35 | kafkaSink.value.send(topic, "key_" + n, "string_" + n)
36 | }
37 | }
38 |
39 | def main (args: Array[String]) {
40 |
41 | val topic = "foo"
42 |
43 | val kafkaServer = new EmbeddedKafkaServer()
44 | kafkaServer.start()
45 | kafkaServer.createTopic(topic, 4)
46 |
47 | val conf = new SparkConf().setAppName("MultipleConsumerGroups").setMaster("local[4]")
48 | val sc = new SparkContext(conf)
49 |
50 | // streams will produce data every second
51 | val ssc = new StreamingContext(sc, Seconds(1))
52 |
53 | val max = 1000
54 |
55 |
56 | //
57 | // the first stream subscribes to consumer group Group1
58 | //
59 |
60 | val props1: Properties = SimpleKafkaClient.getBasicStringStringConsumer(kafkaServer, "Group1")
61 |
62 | val kafkaStream1 =
63 | KafkaUtils.createDirectStream(
64 | ssc,
65 | LocationStrategies.PreferConsistent,
66 | ConsumerStrategies.Subscribe[String, String](
67 | Arrays.asList(topic),
68 | props1.asInstanceOf[java.util.Map[String, Object]]
69 | )
70 |
71 | )
72 |
73 | kafkaStream1.foreachRDD(r => {
74 | println("*** [stream 1] got an RDD, size = " + r.count())
75 | r.foreach(s => println("*** [stream 1] " + s))
76 | if (r.count() > 0) {
77 | println("*** [stream 1] " + r.getNumPartitions + " partitions")
78 | r.glom().foreach(a => println("*** [stream 1] partition size = " + a.size))
79 | }
80 | })
81 |
82 | //
83 | // a second stream, subscribing to the second consumer group (Group2), will
84 | // see all of the same data
85 | //
86 |
87 | val props2: Properties = SimpleKafkaClient.getBasicStringStringConsumer(kafkaServer, "Group2")
88 |
89 | val kafkaStream2 =
90 | KafkaUtils.createDirectStream(
91 | ssc,
92 | LocationStrategies.PreferConsistent,
93 | ConsumerStrategies.Subscribe[String, String](
94 | Arrays.asList(topic),
95 | props2.asInstanceOf[java.util.Map[String, Object]]
96 | )
97 |
98 | )
99 |
100 | kafkaStream2.foreachRDD(r => {
101 | println("*** [stream 2] got an RDD, size = " + r.count())
102 | r.foreach(s => println("*** [stream 2] " + s))
103 | if (r.count() > 0) {
104 | println("*** [stream 2] " + r.getNumPartitions + " partitions")
105 | r.glom().foreach(a => println("*** [stream 2] partition size = " + a.size))
106 | }
107 | })
108 |
109 | ssc.start()
110 |
111 | println("*** started termination monitor")
112 |
113 | // streams seem to need some time to get going
114 | Thread.sleep(5000)
115 |
116 | val producerThread = new Thread("Streaming Termination Controller") {
117 | override def run() {
118 | val client = new SimpleKafkaClient(kafkaServer)
119 |
120 | send(max, sc, topic, client.basicStringStringProducer)
121 | Thread.sleep(5000)
122 | println("*** requesting streaming termination")
123 | ssc.stop(stopSparkContext = false, stopGracefully = true)
124 | }
125 | }
126 | producerThread.start()
127 |
128 | try {
129 | ssc.awaitTermination()
130 | println("*** streaming terminated")
131 | } catch {
132 | case e: Exception => {
133 | println("*** streaming exception caught in monitor thread")
134 | }
135 | }
136 |
137 | // stop Spark
138 | sc.stop()
139 |
140 | // stop Kafka
141 | kafkaServer.stop()
142 |
143 | println("*** done")
144 | }
145 | }
--------------------------------------------------------------------------------
/src/main/scala/Timestamp.scala:
--------------------------------------------------------------------------------
1 | import java.util.{Arrays, Calendar, Properties, TimeZone}
2 |
3 | import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
4 | import org.apache.spark.{SparkConf, SparkContext}
5 | import org.apache.spark.streaming.{Seconds, StreamingContext}
6 | import org.apache.spark.streaming.kafka010.{ConsumerStrategies, KafkaUtils, LocationStrategies}
7 | import util.{EmbeddedKafkaServer, SimpleKafkaClient}
8 |
9 | /**
10 | * Record timestamps were introduced into Kafka 0.10 as described in
11 | * https://cwiki.apache.org/confluence/display/KAFKA/KIP-32+-+Add+timestamps+to+Kafka+message
12 | * and
13 | * https://cwiki.apache.org/confluence/display/KAFKA/KIP-33+-+Add+a+time+based+log+index .
14 | *
15 | * This example sets up two different topics that handle timestamps differently -- topic A has the timestamp
16 | * set by the broker when it receives the record, while topic B passes through the timestamp provided in the record
17 | * (either programmatically when the record was created, as shown here, or otherwise automatically by the producer.)
18 | *
19 | * Since the record carries information about where its timestamp originates, its easy to subscribe to the two topics
20 | * to create a single stream, and then examine the timestamp of every received record and its type.
21 | */
22 | object Timestamp {
23 | def main (args: Array[String]) {
24 |
25 | val topicLogAppendTime = "A"
26 | val topicCreateTime = "B"
27 |
28 | val kafkaServer = new EmbeddedKafkaServer()
29 | kafkaServer.start()
30 | kafkaServer.createTopic(topicLogAppendTime, 4, logAppendTime = true)
31 | kafkaServer.createTopic(topicCreateTime, 4)
32 |
33 | val conf = new SparkConf().setAppName("Timestamp").setMaster("local[4]")
34 | val sc = new SparkContext(conf)
35 |
36 | // streams will produce data every second
37 | val ssc = new StreamingContext(sc, Seconds(1))
38 |
39 | // this many messages
40 | val max = 1000
41 |
42 | // Create the stream.
43 | val props: Properties = SimpleKafkaClient.getBasicStringStringConsumer(kafkaServer)
44 |
45 | val kafkaStream =
46 | KafkaUtils.createDirectStream(
47 | ssc,
48 | LocationStrategies.PreferConsistent,
49 | ConsumerStrategies.Subscribe[String, String](
50 | Arrays.asList(topicLogAppendTime, topicCreateTime),
51 | props.asInstanceOf[java.util.Map[String, Object]]
52 | )
53 |
54 | )
55 |
56 | val timeFormat = new java.text.SimpleDateFormat("HH:mm:ss.SSS")
57 |
58 | // now, whenever this Kafka stream produces data the resulting RDD will be printed
59 | kafkaStream.foreachRDD(r => {
60 | println("*** got an RDD, size = " + r.count())
61 | r.foreach(cr => {
62 |
63 | val time = timeFormat.format(cr.timestamp())
64 | println("Topic [" + cr.topic() + "] Key [" + cr.key + "] Type [" + cr.timestampType().toString +
65 | "] Timestamp [" + time + "]")
66 | })
67 | })
68 |
69 | ssc.start()
70 |
71 | println("*** started termination monitor")
72 |
73 | // streams seem to need some time to get going
74 | Thread.sleep(5000)
75 |
76 | val producerThread = new Thread("Streaming Termination Controller") {
77 | override def run() {
78 | val client = new SimpleKafkaClient(kafkaServer)
79 |
80 | val producer = new KafkaProducer[String, String](client.basicStringStringProducer)
81 |
82 | // the two records are created at almost the same time, so should have similar creation time stamps
83 | // if we didn't provide one, the producer would so so, but then we wouldn't know what it was ...
84 |
85 | val timestamp = Calendar.getInstance().getTime().getTime
86 |
87 | println("Record creation time: " + timeFormat.format(timestamp))
88 |
89 | val record1 = new ProducerRecord(topicLogAppendTime, 1, timestamp, "key1", "value1")
90 | val record2 = new ProducerRecord(topicCreateTime, 1, timestamp, "key2", "value2")
91 |
92 | Thread.sleep(2000)
93 |
94 | // the two records are sent to the Kafka broker two seconds after they are created, and three seconds apart
95 |
96 | producer.send(record1)
97 | Thread.sleep(3000)
98 | producer.send(record2)
99 |
100 | Thread.sleep(5000)
101 | println("*** requesting streaming termination")
102 | ssc.stop(stopSparkContext = false, stopGracefully = true)
103 | }
104 | }
105 | producerThread.start()
106 |
107 | try {
108 | ssc.awaitTermination()
109 | println("*** streaming terminated")
110 | } catch {
111 | case e: Exception => {
112 | println("*** streaming exception caught in monitor thread")
113 | }
114 | }
115 |
116 | // stop Spark
117 | sc.stop()
118 |
119 | // stop Kafka
120 | kafkaServer.stop()
121 |
122 | println("*** done")
123 | }
124 | }
125 |
--------------------------------------------------------------------------------
/src/main/scala/structured/Foreach.scala:
--------------------------------------------------------------------------------
1 | package structured
2 |
3 | import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
4 | import org.apache.spark.sql.{ForeachWriter, Row, SparkSession}
5 | import util.{EmbeddedKafkaServer, SimpleKafkaClient}
6 |
7 | /**
8 | * The 'foreach' operation allows arbitrary computations on the output data in way that is both
9 | * partition-aware (computed on the executors and aware of which partition is being processed) and batch-aware
10 | * (via a separate invocation for each partition/batch combination.)
11 | *
12 | * It is always used by passing the operation an object that implements the 'ForeachWriter' interface. In this
13 | * example, the object doesn't do any "useful" work: instead it is set up to illustrate its slightly arcane state
14 | * management by printing its arguments and state in each of the three overridden methods.
15 | *
16 | * Each instance of ForeachWriter is used for processing a sequence of partition/batch combinations, but at any point
17 | * in time is is setup (via a single open() call) to process one partition/batch combination. Then it gets multiple
18 | * process() calls, providing the the actual data for that partition and batch, and then a single close() call to
19 | * signal that the partition/batch combination has been completely processed.
20 | */
21 | object Foreach {
22 |
23 | def main (args: Array[String]) {
24 |
25 | val topic = "foo"
26 |
27 | println("*** starting Kafka server")
28 | val kafkaServer = new EmbeddedKafkaServer()
29 | kafkaServer.start()
30 | kafkaServer.createTopic(topic, 4)
31 |
32 | Thread.sleep(5000)
33 |
34 | // publish some messages
35 | println("*** Publishing messages")
36 | val messageCount = 16
37 | val client = new SimpleKafkaClient(kafkaServer)
38 | val numbers = 1 to messageCount
39 | val producer = new KafkaProducer[String, String](client.basicStringStringProducer)
40 | numbers.foreach { n =>
41 | producer.send(new ProducerRecord(topic, "[1]key_" + n, "[1]string_" + n))
42 | }
43 | Thread.sleep(5000)
44 |
45 | println("*** Starting to stream")
46 |
47 | val spark = SparkSession
48 | .builder
49 | .appName("Structured_Foreach")
50 | .config("spark.master", "local[4]")
51 | .getOrCreate()
52 |
53 | val ds1 = spark
54 | .readStream
55 | .format("kafka")
56 | .option("kafka.bootstrap.servers", kafkaServer.getKafkaConnect)
57 | .option("subscribe", topic)
58 | .option("startingOffsets", "earliest") // equivalent of auto.offset.reset which is not allowed here
59 | .load()
60 |
61 | val counts = ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
62 |
63 | // process the stream using a custom ForeachWriter that simply prints the data and the state of the ForeachWriter
64 | // in order to illustrate how it works
65 | val query = counts.writeStream
66 | .foreach(new ForeachWriter[Row] {
67 |
68 | // Notice the initialization here is very simple, as it gets called on the driver, but never called
69 | // again on the executor. Any initialization that needs to be called repeatedly on the executor
70 | // needs to go in the open() method.
71 |
72 | // By using an Option, initializing with None and replacing with None in the close() method, we verify that
73 | // process() is only ever called between a matched pair of open() and close() calls.
74 | var myPartition: Option[Long] = None
75 | var myVersion: Option[Long] = None
76 |
77 | /**
78 | * Apart from printing the partition and version, we only accept batches from even numbered partitions.
79 | */
80 | override def open(partitionId: Long, version: Long): Boolean = {
81 | myPartition = Some(partitionId)
82 | myVersion = Some(version)
83 | println(s"*** ForEachWriter: open partition=[$partitionId] version=[$version]")
84 | val processThisOne = partitionId % 2 == 0
85 | // We only accept this partition/batch combination if we return true -- in this case we'll only do so for
86 | // even numbered partitions. This decision could have been based on the version ID as well.
87 | processThisOne
88 | }
89 |
90 | /**
91 | * Since we've saved the partition and batch IDs, we can see which combination each record comes from.
92 | * Notice we only get records from even numbered partitions, since we rejected the odd numbered
93 | * ones in the open() method by returning false.
94 | */
95 | override def process(record: Row) : Unit = {
96 | println(s"*** ForEachWriter: process partition=[$myPartition] version=[$myVersion] record=$record")
97 | }
98 |
99 | /**
100 | * Again we've saved the partition and batch IDs, so we can see which combination is being closed.
101 | * We'll leave error handling for a more advanced example.
102 | */
103 | override def close(errorOrNull: Throwable): Unit = {
104 | println(s"*** ForEachWriter: close partition=[$myPartition] version=[$myVersion]")
105 | myPartition = None
106 | myVersion = None
107 | }
108 | }).start()
109 |
110 | println("*** done setting up streaming")
111 |
112 | Thread.sleep(5000)
113 |
114 | println("*** publishing more messages")
115 | numbers.foreach { n =>
116 | producer.send(new ProducerRecord(topic, "[2]key_" + n, "[2]string_" + n))
117 | }
118 |
119 | Thread.sleep(5000)
120 |
121 | println("*** Stopping stream")
122 | query.stop()
123 |
124 | query.awaitTermination()
125 | spark.stop()
126 | println("*** Streaming terminated")
127 |
128 | // stop Kafka
129 | println("*** Stopping Kafka")
130 | kafkaServer.stop()
131 |
132 | println("*** done")
133 | }
134 | }
--------------------------------------------------------------------------------
/src/main/scala/AddPartitionsWhileStreaming.scala:
--------------------------------------------------------------------------------
1 | import java.util.{Arrays, Properties}
2 |
3 | import org.apache.spark.streaming.kafka010.{ConsumerStrategies, KafkaUtils, LocationStrategies}
4 | import org.apache.spark.streaming.{Seconds, StreamingContext}
5 | import org.apache.spark.{SparkConf, SparkContext}
6 | import util.{EmbeddedKafkaServer, SimpleKafkaClient, SparkKafkaSink}
7 |
8 | /**
9 | * Partitions can be added to a Kafka topic dynamically. This example shows that an existing stream
10 | * will not see the data published to the new partitions, and only when the existing streaming context is terminated
11 | * and a new stream is started from a new context will that data be delivered.
12 | *
13 | * The topic is created with three partitions, and so each RDD the stream produces has three partitions as well,
14 | * even after two more partitions are added to the topic. When a new stream is subsequently created, the RDDs produced
15 | * have five partitions, but only two of them contain data, as all the data has been drained from the initial three
16 | * partitions of the topic, by the first stream.
17 | */
18 | object AddPartitionsWhileStreaming {
19 |
20 | /**
21 | * Publish some data to a topic. Encapsulated here to ensure serializability.
22 | * @param max
23 | * @param sc
24 | * @param topic
25 | * @param config
26 | */
27 | def send(max: Int, sc: SparkContext, topic: String, config: Properties): Unit = {
28 |
29 | // put some data in an RDD and publish to Kafka
30 | val numbers = 1 to max
31 | val numbersRDD = sc.parallelize(numbers, 5)
32 |
33 | val kafkaSink = sc.broadcast(SparkKafkaSink(config))
34 |
35 | println("*** producing data")
36 |
37 | numbersRDD.foreach { n =>
38 | // NOTE:
39 | // 1) the keys and values are strings, which is important when receiving them
40 | // 2) We don't specify which Kafka partition to send to, so a hash of the key
41 | // is used to determine this
42 | kafkaSink.value.send(topic, "key_" + n, "string_" + n)
43 | }
44 | }
45 |
46 | def main (args: Array[String]) {
47 |
48 | val topic = "foo"
49 |
50 | val kafkaServer = new EmbeddedKafkaServer()
51 | kafkaServer.start()
52 | kafkaServer.createTopic(topic, 3)
53 |
54 |
55 |
56 | val conf = new SparkConf().setAppName("AddPartitionsWhileStreaming").setMaster("local[7]")
57 | val sc = new SparkContext(conf)
58 |
59 | // streams will produce data every second
60 | val ssc = new StreamingContext(sc, Seconds(1))
61 |
62 | val max = 500
63 |
64 | val props: Properties = SimpleKafkaClient.getBasicStringStringConsumer(kafkaServer)
65 |
66 | val kafkaStream =
67 | KafkaUtils.createDirectStream(
68 | ssc,
69 | LocationStrategies.PreferConsistent,
70 | ConsumerStrategies.Subscribe[String, String](
71 | Arrays.asList(topic),
72 | props.asInstanceOf[java.util.Map[String, Object]]
73 | )
74 |
75 | )
76 |
77 | // now, whenever this Kafka stream produces data the resulting RDD will be printed
78 | kafkaStream.foreachRDD(r => {
79 | println("[1] *** got an RDD, size = " + r.count())
80 | r.foreach(s => println(s))
81 | if (r.count() > 0) {
82 | // let's see how many partitions the resulting RDD has -- notice that it has nothing
83 | // to do with the number of partitions in the RDD used to publish the data (4), nor
84 | // the number of partitions of the topic (which also happens to be four.)
85 | println("[1] *** " + r.getNumPartitions + " partitions")
86 | r.glom().foreach(a => println("[1] *** partition size = " + a.size))
87 | }
88 | })
89 |
90 | ssc.start()
91 |
92 | println("*** started streaming context")
93 |
94 | // streams seem to need some time to get going
95 | Thread.sleep(5000)
96 |
97 |
98 | val client = new SimpleKafkaClient(kafkaServer)
99 |
100 | send(max, sc, topic, client.basicStringStringProducer)
101 | Thread.sleep(5000)
102 |
103 | println("*** adding partitions to topic")
104 |
105 | kafkaServer.addPartitions(topic, 5)
106 |
107 | Thread.sleep(5000)
108 |
109 | send(max, sc, topic, client.basicStringStringProducer)
110 |
111 | Thread.sleep(5000)
112 |
113 | println("*** stop first streaming context")
114 | ssc.stop(stopSparkContext = false)
115 | try {
116 | ssc.awaitTermination()
117 | println("*** streaming terminated for the first time")
118 | } catch {
119 | case e: Exception => {
120 | println("*** streaming exception caught in monitor thread (first context)")
121 | }
122 | }
123 |
124 | println("*** create second streaming context")
125 | val ssc2 = new StreamingContext(sc, Seconds(1))
126 |
127 | println("*** create a second stream from the second streaming context")
128 | val kafkaStream2 =
129 | KafkaUtils.createDirectStream(
130 | ssc2,
131 | LocationStrategies.PreferConsistent,
132 | ConsumerStrategies.Subscribe[String, String](
133 | Arrays.asList(topic),
134 | props.asInstanceOf[java.util.Map[String, Object]]
135 | )
136 |
137 | )
138 |
139 | kafkaStream2.foreachRDD(r => {
140 | println("[2] *** got an RDD, size = " + r.count())
141 | r.foreach(s => println(s))
142 | if (r.count() > 0) {
143 | // let's see how many partitions the resulting RDD has -- notice that it has nothing
144 | // to do with the number of partitions in the RDD used to publish the data (4), nor
145 | // the number of partitions of the topic (which also happens to be four.)
146 | println("[2] *** " + r.getNumPartitions + " partitions")
147 | r.glom().foreach(a => println("[2] *** partition size = " + a.size))
148 | }
149 | })
150 |
151 | println("*** start second streaming context")
152 | ssc2.start()
153 |
154 | Thread.sleep(5000)
155 |
156 | println("*** requesting streaming termination")
157 | ssc2.stop(stopSparkContext = false, stopGracefully = true)
158 |
159 |
160 | try {
161 | ssc2.awaitTermination()
162 | println("*** streaming terminated")
163 | } catch {
164 | case e: Exception => {
165 | println("*** streaming exception caught in monitor thread")
166 | }
167 | }
168 |
169 | // stop Spark
170 | sc.stop()
171 |
172 | // stop Kafka
173 | kafkaServer.stop()
174 |
175 | println("*** done")
176 | }
177 | }
--------------------------------------------------------------------------------
/src/main/scala/applications/stock_price_feed/StockMarketData.scala:
--------------------------------------------------------------------------------
1 | package applications.stock_price_feed
2 |
3 | import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
4 | import java.util.{Arrays, Properties}
5 |
6 | import org.apache.kafka.clients.consumer.ConsumerConfig
7 | import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
8 | import org.apache.kafka.common.serialization.{Deserializer, Serializer}
9 | import org.apache.spark.streaming.kafka010.{ConsumerStrategies, KafkaUtils, LocationStrategies}
10 | import org.apache.spark.streaming.{Seconds, StreamingContext}
11 | import org.apache.spark.{SparkConf, SparkContext}
12 | import util.{EmbeddedKafkaServer, PartitionMapAnalyzer, SimpleKafkaClient}
13 |
14 | import scala.collection.{Iterator, mutable}
15 |
16 | class TradeData(val symbol: String, val price: Double, val volume: Long) extends Serializable {
17 |
18 | }
19 |
20 | class ChunkedTradeData(val symbol: String) extends Serializable {
21 | var trades = 0
22 | var totalAmount = 0.0
23 | var totalVolume: Long = 0
24 |
25 | def addTrade(trade: TradeData) : Unit = {
26 | trades = trades + 1
27 | totalVolume = totalVolume + trade.volume
28 | totalAmount = totalAmount + trade.volume * trade.price
29 | }
30 |
31 | def averagePrice = totalAmount / totalVolume
32 | }
33 |
34 | class TradeDataSerializer extends Serializer[TradeData] {
35 |
36 | override def close(): Unit = {}
37 |
38 | override def configure(config: java.util.Map[String, _], isKey: Boolean) : Unit = {}
39 |
40 | override def serialize(topic: String, data: TradeData) : Array[Byte] = {
41 | val stream: ByteArrayOutputStream = new ByteArrayOutputStream()
42 | val oos = new ObjectOutputStream(stream)
43 | oos.writeObject(data)
44 | oos.close
45 | stream.toByteArray
46 | }
47 | }
48 |
49 | class TradeDataDeserializer extends Deserializer[TradeData] {
50 |
51 | override def close(): Unit = {}
52 |
53 | override def configure(config: java.util.Map[String, _], isKey: Boolean) : Unit = {}
54 |
55 | override def deserialize(topic: String, data: Array[Byte]) : TradeData = {
56 | val ois = new ObjectInputStream(new ByteArrayInputStream(data))
57 | val value = ois.readObject
58 | ois.close
59 | value.asInstanceOf[TradeData]
60 | }
61 | }
62 |
63 |
64 |
65 | object StockMarketData {
66 |
67 | def getProducer(server: EmbeddedKafkaServer) : Properties = {
68 | val config: Properties = new Properties
69 | config.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[TradeDataSerializer].getCanonicalName)
70 | config.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[TradeDataSerializer].getCanonicalName)
71 | config.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, server.getKafkaConnect)
72 | config
73 | }
74 |
75 | def getConsumer(server: EmbeddedKafkaServer, group:String = "MyGroup") : Properties = {
76 | val consumerConfig: Properties = new Properties
77 | consumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, group)
78 | consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[TradeDataDeserializer].getCanonicalName)
79 | consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[TradeDataDeserializer].getCanonicalName)
80 | consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, server.getKafkaConnect)
81 | consumerConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
82 | consumerConfig
83 | }
84 |
85 | def main (args: Array[String]) {
86 |
87 | val topic1 = "SYM1"
88 | val topic2 = "SYM2"
89 |
90 | // topics are partitioned differently
91 | val kafkaServer = new EmbeddedKafkaServer()
92 | kafkaServer.start()
93 | kafkaServer.createTopic(topic1, 1)
94 | kafkaServer.createTopic(topic2, 1)
95 |
96 | val conf = new SparkConf().setAppName("StockMarketData").setMaster("local[10]")
97 | val sc = new SparkContext(conf)
98 |
99 | // streams will produce data every second
100 | val ssc = new StreamingContext(sc, Seconds(1))
101 |
102 | // this many messages
103 | val max = 100
104 |
105 | // Create the stream.
106 | val props: Properties = getConsumer(kafkaServer)
107 |
108 | val rawDataFeed =
109 | KafkaUtils.createDirectStream(
110 | ssc,
111 | LocationStrategies.PreferConsistent,
112 | ConsumerStrategies.Subscribe[String, TradeData](
113 | Arrays.asList(topic1, topic2),
114 | props.asInstanceOf[java.util.Map[String, Object]]
115 | )
116 |
117 | )
118 |
119 | // now, whenever this Kafka stream produces data the resulting RDD will be printed
120 | rawDataFeed.foreachRDD(r => {
121 | println("*** got an RDD, size = " + r.count())
122 |
123 | PartitionMapAnalyzer.analyze(r)
124 |
125 | })
126 |
127 | def chunkingFunc(i: Iterator[TradeData]) : Iterator[Map[String, ChunkedTradeData]] = {
128 | val m = new mutable.HashMap[String, ChunkedTradeData]()
129 | i.foreach {
130 | case trade: TradeData =>
131 | if (m.contains(trade.symbol)) {
132 | m(trade.symbol).addTrade(trade)
133 | } else {
134 | val chunked = new ChunkedTradeData(trade.symbol)
135 | chunked.addTrade(trade)
136 | m(trade.symbol) = chunked
137 | }
138 | }
139 | Iterator.single(m.toMap)
140 | }
141 |
142 | val decodedFeed = rawDataFeed.map(cr => cr.value())
143 |
144 | val chunkedDataFeed = decodedFeed.mapPartitions(chunkingFunc, preservePartitioning = true)
145 |
146 | chunkedDataFeed.foreachRDD(rdd => {
147 | rdd.foreach(m =>
148 | m.foreach {
149 | case (symbol, chunk) =>
150 | println(s"Symbol ${chunk.symbol} Price ${chunk.averagePrice} Volume ${chunk.totalVolume} Trades ${chunk.trades}")
151 | })
152 | })
153 |
154 | ssc.start()
155 |
156 | println("*** started streaming context")
157 |
158 | // streams seem to need some time to get going
159 | Thread.sleep(5000)
160 |
161 | val producerThreadTopic1 = new Thread("Producer thread 1") {
162 | override def run() {
163 | val client = new SimpleKafkaClient(kafkaServer)
164 |
165 | val numbers = 1 to max
166 |
167 | val producer = new KafkaProducer[String, TradeData](getProducer(kafkaServer))
168 |
169 | numbers.foreach { n =>
170 | // NOTE:
171 | // 1) the keys and values are strings, which is important when receiving them
172 | // 2) We don't specify which Kafka partition to send to, so a hash of the key
173 | // is used to determine this
174 | producer.send(new ProducerRecord(topic1, new TradeData("SYM1", 12.0, 100)))
175 | }
176 |
177 | }
178 | }
179 |
180 | val producerThreadTopic2 = new Thread("Producer thread 2; controlling termination") {
181 | override def run() {
182 | val client = new SimpleKafkaClient(kafkaServer)
183 |
184 | val numbers = 1 to max
185 |
186 | val producer = new KafkaProducer[String, TradeData](getProducer(kafkaServer))
187 |
188 | numbers.foreach { n =>
189 | // NOTE:
190 | // 1) the keys and values are strings, which is important when receiving them
191 | // 2) We don't specify which Kafka partition to send to, so a hash of the key
192 | // is used to determine this
193 | producer.send(new ProducerRecord(topic2, new TradeData("SYM2", 123.0, 200)))
194 | }
195 | Thread.sleep(10000)
196 | println("*** requesting streaming termination")
197 | ssc.stop(stopSparkContext = false, stopGracefully = true)
198 | }
199 | }
200 |
201 | producerThreadTopic1.start()
202 | producerThreadTopic2.start()
203 |
204 | try {
205 | ssc.awaitTermination()
206 | println("*** streaming terminated")
207 | } catch {
208 | case e: Exception => {
209 | println("*** streaming exception caught in monitor thread")
210 | }
211 | }
212 |
213 | // stop Spark
214 | sc.stop()
215 |
216 | // stop Kafka
217 | kafkaServer.stop()
218 |
219 | println("*** done")
220 | }
221 | }
222 |
--------------------------------------------------------------------------------
/.idea/uiDesigner.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 |
6 |
7 |
8 |
9 |
10 |
11 |
12 |
13 |
14 |
15 |
16 |
17 |
18 |
19 |
20 |
21 |
22 |
23 |
24 |
25 |
26 |
27 |
28 |
29 |
30 |
31 |
32 |
33 |
34 |
35 |
36 |
37 |
38 |
39 |
40 |
41 |
42 |
43 |
44 |
45 |
46 |
47 |
48 |
49 |
50 |
51 |
52 |
53 |
54 |
55 |
56 |
57 |
58 |
59 |
60 |
61 |
62 |
63 |
64 |
65 |
66 |
67 |
68 |
69 |
70 |
71 |
72 |
73 |
74 |
75 |
76 |
77 |
78 |
79 |
80 |
81 |
82 |
83 |
84 |
85 |
86 |
87 |
88 |
89 |
90 |
91 |
92 |
93 |
94 |
95 |
96 |
97 |
98 |
99 |
100 |
101 |
102 |
103 |
104 |
105 |
106 |
107 |
108 |
109 |
110 |
111 |
112 |
113 |
114 |
115 |
116 |
117 |
118 |
119 |
120 |
121 |
122 |
123 |
124 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # Self-contained examples of Spark streaming integrated with Kafka
2 |
3 | [](https://travis-ci.org/spirom/spark-streaming-with-kafka)
4 |
5 | The goal of this project is to make it easy to experiment with Spark Streaming based on Kafka,
6 | by creating examples that run against an embedded Kafka server and an embedded Spark instance.
7 | Of course, in making everything easy to work with we also make it perform poorly. It would be a
8 | really bad idea to try to learn anything about performance from this project: it's all
9 | about functionality, although we sometimes get insight into performance issues by understanding
10 | the way the
11 | code interacts with RDD partitioning in Spark and topic partitioning in Kafka.
12 |
13 | ## Related projects
14 |
15 | This project is derived from the
16 | [LearningSpark project](https://github.com/spirom/LearningSpark) which explores the full range of Spark APIs from the
17 | viewpoint of Scala developers. There is a corresponding, but much less comprehensive Java version at
18 | [learning-spark-with-java](https://github.com/spirom/learning-spark-with-java).
19 |
20 | The [spark-data-sources](https://github.com/spirom/spark-data-sources) project is focused on
21 | the new experimental APIs introduced in Spark 2.3.0 for developing adapters for
22 | external data sources of
23 | various kinds. This API is essentially a Java API (developed in Java) to avoid forcing
24 | developers to adopt Scala for their data source adapters. Consequently, the example data sources
25 | in this project are written in Java, but both Java and Scala usage examples are provided.
26 |
27 | ## Dependencies
28 |
29 | The project was created with IntelliJ Idea 14 Community Edition. It is known to work with
30 | JDK 1.8, Scala 2.11.12, and Spark 2.3.0 with its Kafka 0.10 shim library on Ubuntu Linux.
31 |
32 | It uses the Direct DStream package spark-streaming-kafka-0-10 for
33 | Spark Streaming integration with Kafka 0.10.0.1. The details behind this are explained in the
34 | [Spark 2.3.0 documentation](https://spark.apache.org/docs/2.3.0/streaming-kafka-integration.html).
35 |
36 | Note that, with the release of Spark 2.3.0, the formerly stable Receiver DStream APIs are now deprecated, and the
37 | formerly experimental Direct DStream APIs are now stable.
38 |
39 | ## Using the deprecated (Receiver DStream) Kafka 0.8.0 APIs
40 |
41 | I've kept around the examples for the older, stable Kafka integration on the
42 | [kafka0.8](https://github.com/spirom/spark-streaming-with-kafka/tree/kafka0.8) branch
43 |
44 | # Structured Streaming
45 |
46 | There's a [separate set of examples](src/main/scala/structured) for
47 | Kafka integration with the new Structured Streaming features (mainstream as of Spark 2.2).
48 |
49 | ## Utilities
50 |
51 |
Support for understanding how subscribed Kafka topics and their Kafka partitions map to partitions in the
68 | RDD that is emitted by the Spark stream.
Show how call to awaitTermination() throws propagated exceptions.
96 |
97 |
98 |
99 | ## Partitioning Examples
100 |
101 | Partitioning is an important factor in determining the scalability oif Kafka-based streaming applications.
102 | In this set of examples you can see the relationship between a number of facets of partitioning.
103 | * The number of partitions in the RDD that is being published to a topic -- if indeed this involves an RDD, as the data is often published from a non-Spark application
104 | * The number of partitions of the topic itself (usually specified at topic creation)
105 | * THe number of partitions in the RDDs created by the Kafka stream
106 | * Whether and how messages move between partitions when they are transferred
107 |
108 | When running these examples, look for:
109 | * The topic partition number that is printed with each ConsumerRecord
110 | * After all the records are printed, the number of partitions in the resulting RDD and size of each partition. For example:
111 | *** 4 partitions
112 | *** partition size = 253
113 | *** partition size = 252
114 | *** partition size = 258
115 | *** partition size = 237
116 |
117 |
118 | Another way these examples differ from the basic examples above is that Spark is used to publish to the topic.
119 | Perhaps surprisingly, this is not completely straightforward, and relies on [util/SparkKafkaSink.scala](src/main/scala/util/SparkKafkaSink.scala).
120 | An alternative approach to this can be found [here](https://docs.cloud.databricks.com/docs/latest/databricks_guide/07%20Spark%20Streaming/09%20Write%20Output%20To%20Kafka.html).
121 |
122 |
123 |
Partitions can be added to a Kafka topic dynamically. This example shows that an existing stream
140 | will not see the data published to the new partitions, and only when the existing streaming context is terminated
141 | and a new stream is started from a new context will that data be delivered.
142 |
143 | The topic is created with three partitions, and so each RDD the stream produces has three partitions as well,
144 | even after two more partitions are added to the topic. This is what's received after the first 500 records
145 | are published to the topic while it has only three partitions:
When two partitions are added and another 500 messages are published, this is what's received
154 | (note both the number of partitions and the number of messages):
When a new stream is subsequently created, the RDDs produced
164 | have five partitions, but only two of them contain data, as all the data has been drained from the initial three
165 | partitions of the topic, by the first stream. Now all 500 messages (288 + 212) from the second set have been delivered.
Two streams subscribing to the same topic via a single consumer group divide up the data.
192 | There's an interesting partitioning interaction here as the streams each get data from two fo the four topic
193 | partitions, and each produce RDDs with two partitions each.
Record timestamps were introduced into Kafka 0.10 as described in
231 | KIP-32
232 | and
233 | KIP-33.
234 |
235 |
This example sets up two different topics that handle timestamps differently -- topic A has the timestamp
236 | set by the broker when it receives the record, while topic B passes through the timestamp provided in the record
237 | (either programmatically when the record was created, as shown here, or otherwise automatically by the producer.)
238 |
239 |
Since the record carries information about where its timestamp originates, its easy to subscribe to the two topics
240 | to create a single stream, and then examine the timestamp of every received record and its type.
241 |
242 |
NOTE: The use of timestamps to filter topics in the broker, as introduced in Kafka 0.10.1, is blocked on
243 | SPARK-18057.