("ufo_sightings", key, msg);
45 | producer.send(data);
46 | long wait = Math.round(Math.random() * 25);
47 | Thread.sleep(wait);
48 | }
49 | } catch (Exception e) {
50 | e.printStackTrace();
51 | } finally {
52 | producer.close();
53 | }
54 | }
55 | }
56 |
--------------------------------------------------------------------------------
/SimpleFlafka/README.md:
--------------------------------------------------------------------------------
1 | # Simple Flafka
2 |
3 | This contains the sample code for the "Scalability of Kafka Messaging using Consumer Groups"
4 |
5 | ## Instructions
6 |
7 | First, run the Flume agent:
8 |
9 |
10 | $ sudo -u hdfs flume-ng agent -n flume/TwitterAgent -f etc/twitter.conf --conf etc/flume-ng/conf/
11 |
12 |
13 | Then, you can use the kafka-console-consumer command to begin consuming the topic.
14 |
15 |
16 | $ cp /etc/conf/tools-log4j.properties consumer.properties
17 | $ kafka-console-consumer --zookeeper :2181 --topic "tweets" --consumer.config consumer.properties
18 |
19 |
--------------------------------------------------------------------------------
/SimpleFlafka/etc/twitter.conf:
--------------------------------------------------------------------------------
1 | #Naming the components of the current agent:
2 |
3 | TwitterAgent.sources= Twitter
4 | TwitterAgent.channels= MemChannel
5 | TwitterAgent.sinks = Kafkanic
6 |
7 | # Describing/Configuring the source
8 | TwitterAgent.sources.Twitter.type = org.apache.flume.source.twitter.TwitterSource
9 | TwitterAgent.sources.Twitter.consumerKey =
10 | TwitterAgent.sources.Twitter.consumerSecret =
11 | TwitterAgent.sources.Twitter.accessToken =
12 | TwitterAgent.sources.Twitter.accessTokenSecret =
13 | TwitterAgent.sources.Twitter.keywords = iphonex, teampixel, samsungs8, note8
14 | TwitterAgent.sources.Twitter.language= en, en-gb
15 |
16 | # Describing/Configuring the sink
17 |
18 | TwitterAgent.sinks.Kafkanic.type = org.apache.flume.sink.kafka.KafkaSink
19 | TwitterAgent.sinks.Kafkanic.channel = MemChannel
20 | TwitterAgent.sinks.Kafkanic.brokerList = sgostest-1.gce.cloudera.com:9092,sgostest-2.gce.cloudera.com:9092
21 | TwitterAgent.sinks.Kafkanic.batchSize =100
22 | TwitterAgent.sinks.Kafkanic.topic = tweets_partitioned
23 |
24 | # Describing/Configuring the channel TwitterAgent.channels.MemChannel.type = memory
25 | TwitterAgent.channels.MemChannel.capacity = 10000
26 | TwitterAgent.channels.MemChannel.transactionCapacity = 100
27 | TwitterAgent.channels.MemChannel.type = memory
28 |
29 | # Binding the source and sink to the channel
30 | TwitterAgent.sources.Twitter.channels = MemChannel
--------------------------------------------------------------------------------
/SimpleFlafka/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 | 4.0.0
4 | simpleFlafka
5 | simpleFlafka
6 | 1.0-SNAPSHOT
7 |
8 |
9 | cloudera
10 | https://repository.cloudera.com/artifactory/cloudera-repos/
11 |
12 |
13 |
14 |
15 | org.apache.kafka
16 | kafka-clients
17 | 1.0.1-kafka-3.1.0
18 | compile
19 |
20 |
21 |
22 |
23 |
24 | org.apache.maven.plugins
25 | maven-compiler-plugin
26 | 3.7.0
27 |
28 | 1.8
29 | 1.8
30 |
31 |
32 |
33 |
34 |
35 |
--------------------------------------------------------------------------------
/SimpleFlafka/src/main/java/FlafkaConsumer.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2018
3 | */
4 | import java.util.Arrays;
5 | import java.util.Properties;
6 |
7 | import org.apache.kafka.clients.consumer.ConsumerRecord;
8 | import org.apache.kafka.clients.consumer.ConsumerRecords;
9 | import org.apache.kafka.clients.consumer.KafkaConsumer;
10 |
11 | public class FlafkaConsumer {
12 |
13 | public static void main(String[] args){
14 |
15 | Properties props = new Properties();
16 | props.put("bootstrap.servers","broker-1.gce.cloudera.com:9092, broker-2.gce.cloudera.com:9092");
17 | props.put("group.id",args[0]);
18 | props.put("enable.auto.commit","true");
19 | props.put("auto.commit.interval.ms","1000");
20 | props.put("key.deserializer","org.apache.kafka.common.serialization.StringDeserializer");
21 | props.put("value.deserializer","org.apache.kafka.common.serialization.StringDeserializer");
22 |
23 | KafkaConsumer consumer = new KafkaConsumer(props);
24 | String topic = args[1];
25 |
26 | consumer.subscribe(Arrays.asList(topic));
27 | while(true){
28 | ConsumerRecords records = consumer.poll(100);
29 | for(ConsumerRecord record : records ) {
30 | System.out.printf("offset = %d, key = %s \n",record.offset(),record.value().split(",")[4]);
31 | // System.out.printf("offset = %d, key = %s, value = %s \n",record.offset(),record.key(),record.value());
32 |
33 | }
34 | }
35 | }
36 | }
37 |
--------------------------------------------------------------------------------
/SimpleFlafka/src/main/java/META-INF/MANIFEST.MF:
--------------------------------------------------------------------------------
1 | Manifest-Version: 1.0
2 | Main-Class: FlafkaConsumer
3 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/README.md:
--------------------------------------------------------------------------------
1 | _Copyright © Cloudera, Inc. 2018_
2 | # Spark Structured Streaming reference application for CDH
3 |
4 | ## Introduction
5 |
6 | This project includes sample applications that demonstrate an Apache Kafka -> Apache Spark Structured Streaming -> Apache Kudu pipeline for ingestion.
7 |
8 | Please check out the [documentation](docs/doc.md) to get an overview of building Spark structured streaming applications on the CDH platform,
9 | a description of the use case the application solves, the components and the integration techniques used to realize a simple
10 | streaming system using the Cloudera stack.
11 |
12 | There are two sample applications implementing a streaming application in two different ways.
13 |
14 | * The simpleApp focuses on the integration aspect:
15 | it demonstrates the simplest way to connect Spark with Kafka and Kudu.
16 | * The advancedApp also shows a way to abstract out the business logic from the application.
17 | It enables easy switching between various sources and sinks and eases testing on different levels.
18 |
19 | The applications demonstrate some basic Structured Streaming techniques, including stream - static table join to enrich data in the incoming stream
20 | and windowing.
21 |
22 | For the preparation and execution instructions please see the README file of the separate projects.
23 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/README.md:
--------------------------------------------------------------------------------
1 | _Copyright © Cloudera, Inc. 2019_
2 | # Executing the application
3 | ## Locally
4 |
5 | com.cloudera.streaming.refapp.StructuredStreams inputDir outputDir kudu-master
6 |
7 | It will start an embedded Kafka and Spark instance.
8 | It's intended to be used during development and testing.
9 |
10 | This version can read local json files or generated input for streams and local files
11 | or Kudu tables for the static datasets.
12 | It prodcues CSV output or writes to Kudu. You can easily change any sink or source in the code of StructuredStreams.
13 |
14 | For input you can use samples in the src/test/resources/samples directory of this project,
15 | or your own files organized to the same structure.
16 |
17 | As an alternative you can use a data generator that will keep producing randomized vendor, customer and
18 | transaction records until it is stopped.
19 | To use the data generator add / uncomment the following section to the constructor call of the
20 | Application in the StructuredStreams object:
21 | ```
22 | initSources = {
23 | CustomerGenerator(kafkaConfig, "customer").start()
24 | VendorGenerator(kafkaConfig, "vendor").start()
25 | TransactionGenerator(kafkaConfig, "transaction").start()
26 | },
27 | ```
28 |
29 |
30 | ## Submitting to Spark service running on a cluster
31 |
32 | ### Cluster requriements
33 | The application can be deployed on a cluster that already has all the required services:
34 | * Kafka
35 | * Spark
36 | * Kudu
37 |
38 | their dependencies:
39 | * Zookeeper
40 | * HDFS
41 |
42 | #### Secured cluster
43 | * Kerberos (the application is tested with MIT Kerberos, AD should also be ok)
44 | * Sentry
45 |
46 | All the other services should be configured to use SSL/TLS, Kerberos for authentication and Sentry for authorization
47 |
48 | The application itself does not require Impala but it is used by the init scripts to create the Kudu tables and insert sample initial data.
49 |
50 | ### Preparations
51 |
52 | 1. Execute
53 |
54 | `mvn clean package`
55 |
56 | 2. Copy the target/streaming-ref-app-advanced-0.1-SNAPSHOT-jar-with-dependencies.jar, db/init_kudu_db.sql and all files
57 | from the ../scripts directory to a host on the cluster.
58 | 3. Ssh to that host
59 | 4. Edit config.sh. It contains reasonable defaults, make sure that you set each value fitting to your environment.
60 | 5. Execute all the other .sh files. They will generate various config files used by the application:
61 | 1. `kudu.sh` - creates the streaming_ref datbase and the tables in Kudu and sentry privileges required to access them
62 | 2. `producer.sh` - creates sentry privileges and configuration files for the application that generates
63 | input records and sends them to Kafka
64 | 3. `spark-kafka.sh` - creates Kafka related sentry privileges and configuration files for Spark application
65 | 4. `topics.sh` - creates the Kafka topics
66 |
67 |
68 | ### Using the application
69 | The DeployedStructuredStreams application will read records from 3 Kafka topics (customer, vendor, and transaction) and it will
70 | write customer, vendor and transaction data to Kudu tables (customers, vendors, states, valid_transactions, invalid_transactions,
71 | customer_orphans, vendor_orphans, transactions_operational_metadata).
72 |
73 | When the application is started / submitted to the Spark service it will start the streaming pipeline,
74 | but it will not produce any output until it gets data from the Kafka topic.
75 |
76 | You can send data on your own (e.g. using a ConsoleProducer to producer records with the same JSON format as the sample files in
77 | src/test/resources/samples/kafka) or you can use the DeployedDataGenerator application.
78 |
79 | You can check the output e.g. by using Impala.
80 | * Authenticate with `kinit` using a user that has access to the streaming_ref database and all the tables you want to check
81 | * `impala-shell -i -k --ssl`
82 | * in the shell execute
83 | ```
84 | use streaming_ref;
85 |
86 | select 'valid_transactions' as table_name, count(*) from valid_transactions
87 | union
88 | select 'invalid_transactions', count(*) from invalid_transactions
89 | union
90 | select 'customer_orphans', count(*) from customer_orphans
91 | union
92 | select 'vendor_orphans', count(*) from vendor_orphans
93 | union
94 | select 'transactions_operational_metadata', count(*) from transactions_operational_metadata;
95 | ```
96 | to quickly check if the application is producing output, or you can execute any other queries against the output tables.
97 |
98 |
99 | #### Submitting the application without security
100 |
101 | Execute
102 |
103 | ```
104 | spark-submit --files consumer.properties \
105 | --class com.cloudera.streaming.refapp.DeployedStructuredStreams --deploy-mode cluster \
106 | --master yarn streaming-ref-app-advanced-0.1-SNAPSHOT-jar-with-dependencies.jar \
107 | consumer.properties
108 | ```
109 |
110 | #### Submitting the application on a secured cluster
111 |
112 | Execute
113 |
114 | ```
115 | kinit -kt
116 |
117 | spark-submit --files consumer.properties,kafka_client_jaas.conf, --driver-java-options \
118 | "-Djava.security.auth.login.config=./kafka_client_jaas.conf" --class com.cloudera.streaming.refapp.DeployedStructuredStreams \
119 | --conf "spark.executor.extraJavaOptions=-Djava.security.auth.login.config=./kafka_client_jaas.conf" \
120 | --deploy-mode cluster --master yarn streaming-ref-app-advanced-0.1-SNAPSHOT-jar-with-dependencies.jar \
121 | consumer.properties
122 | ```
123 |
124 | By default the application will keep running until you kill it in yarn. You can use an additional timeToLive parameter when the
125 | application is submitted, in this case the application will stop after the given time (in seconds). E.g.
126 | ```
127 | spark-submit...streaming-ref-app-advanced-0.1-SNAPSHOT-jar-with-dependencies.jar consumer.properties 600
128 | ```
129 | will stop after 10 minutes.
130 |
131 | ### Running the data generator
132 | Start `java -cp streaming-ref-app-advanced-0.1-SNAPSHOT-jar-with-dependencies.jar \
133 | com.cloudera.streaming.refapp.DeployedDataGenerator producer.properties`.
134 |
135 | # Testing
136 | TransactionsFlowUnitTest and LocalIntegrationTest demonstrate how to write unit tests and integration tests
137 | for Spark Structured Streaming applications.
138 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/db/init_kudu_db.sql:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 |
5 | CREATE DATABASE IF NOT EXISTS streaming_ref;
6 | USE streaming_ref;
7 |
8 | DROP TABLE IF EXISTS customers;
9 | CREATE TABLE customers (
10 | customer_id INT PRIMARY KEY,
11 | first_name STRING,
12 | last_name STRING,
13 | state_name STRING,
14 | state_abbreviation STRING,
15 | update_timestamp TIMESTAMP)
16 | PARTITION BY HASH (customer_id) PARTITIONS 10
17 | STORED AS KUDU TBLPROPERTIES ('kudu.num_tablet_replicas' = '3');
18 |
19 | DROP TABLE IF EXISTS vendors;
20 | CREATE TABLE vendors (
21 | vendor_id INT PRIMARY KEY,
22 | vendor_name STRING,
23 | phone_number STRING,
24 | update_timestamp TIMESTAMP)
25 | PARTITION BY HASH (vendor_id) PARTITIONS 10
26 | STORED AS KUDU TBLPROPERTIES ('kudu.num_tablet_replicas' = '3');
27 |
28 | DROP TABLE IF EXISTS states;
29 | CREATE TABLE states (
30 | state_id INT PRIMARY KEY,
31 | state_name STRING,
32 | state_abbreviation STRING)
33 | STORED AS KUDU TBLPROPERTIES ('kudu.num_tablet_replicas' = '3');
34 |
35 | DROP TABLE IF EXISTS valid_transactions;
36 | CREATE TABLE valid_transactions (
37 | event_timestamp TIMESTAMP,
38 | transaction_id STRING,
39 | customer_id INT,
40 | vendor_id INT,
41 | event_state STRING,
42 | price STRING,
43 | card_type STRING,
44 | PRIMARY KEY (event_timestamp, transaction_id)
45 | )
46 | PARTITION BY
47 | HASH (transaction_id) PARTITIONS 15,
48 | RANGE (event_timestamp)
49 | (PARTITION '2018-11-01' <= VALUES < '2018-12-01')
50 | STORED AS KUDU TBLPROPERTIES ('kudu.num_tablet_replicas' = '3');
51 | ALTER TABLE valid_transactions ADD RANGE PARTITION '2018-12-01' <= VALUES < '2019-01-01';
52 | ALTER TABLE valid_transactions ADD RANGE PARTITION '2019-01-01' <= VALUES < '2019-02-01';
53 | ALTER TABLE valid_transactions ADD RANGE PARTITION '2019-02-01' <= VALUES < '2019-03-01';
54 | -- ...
55 |
56 | DROP TABLE IF EXISTS invalid_transactions;
57 | CREATE TABLE invalid_transactions (
58 | transaction_id STRING PRIMARY KEY,
59 | customer_id INT,
60 | vendor_id INT,
61 | event_state STRING,
62 | event_timestamp TIMESTAMP,
63 | price STRING,
64 | card_type STRING)
65 | PARTITION BY
66 | HASH (transaction_id) PARTITIONS 15
67 | STORED AS KUDU TBLPROPERTIES ('kudu.num_tablet_replicas' = '3');
68 |
69 | DROP TABLE IF EXISTS customer_orphans;
70 | CREATE TABLE customer_orphans (
71 | customer_id INT PRIMARY KEY,
72 | first_name STRING,
73 | last_name STRING,
74 | state_id INT,
75 | update_timestamp TIMESTAMP)
76 | STORED AS KUDU TBLPROPERTIES ('kudu.num_tablet_replicas' = '3');
77 |
78 | DROP TABLE IF EXISTS vendor_orphans;
79 | CREATE TABLE vendor_orphans (
80 | vendor_id INT PRIMARY KEY,
81 | vendor_name STRING,
82 | phone_number STRING,
83 | update_timestamp TIMESTAMP)
84 | STORED AS KUDU TBLPROPERTIES ('kudu.num_tablet_replicas' = '3');
85 |
86 | DROP TABLE IF EXISTS transactions_operational_metadata;
87 | CREATE TABLE transactions_operational_metadata(
88 | start_ts TIMESTAMP PRIMARY KEY,
89 | end_ts TIMESTAMP,
90 | num_transactions BIGINT)
91 | STORED AS KUDU TBLPROPERTIES ('kudu.num_tablet_replicas' = '3');
92 |
93 | insert into customers values (1, 'John', 'Doe', 'Alabama', 'AL', '2018-01-01');
94 | insert into customers values (2, 'Jane', 'Miller', 'Alaska', 'AK', '2018-01-01');
95 |
96 | insert into vendors values (1, 'Apple', '123456', '2018-11-13');
97 | insert into vendors values (2, 'Dell', '345678', '2018-11-13');
98 |
99 | INSERT INTO states values (-1, 'Unknown', '??');
100 | INSERT INTO states values (1, 'Alabama', 'AL');
101 | INSERT INTO states values (2, 'Alaska', 'AK');
102 | INSERT INTO states values (3, 'Arizona', 'AZ');
103 | INSERT INTO states values (4, 'Arkansas', 'AR');
104 | INSERT INTO states values (5, 'California', 'CA');
105 | INSERT INTO states values (6, 'Colorado', 'CO');
106 | INSERT INTO states values (7, 'Connecticut', 'CT');
107 | INSERT INTO states values (8, 'Delaware', 'DE');
108 | INSERT INTO states values (9, 'District of Columbia', 'DC');
109 | INSERT INTO states values (10, 'Florida', 'FL');
110 | INSERT INTO states values (11, 'Georgia', 'GA');
111 | INSERT INTO states values (12, 'Hawaii', 'HI');
112 | INSERT INTO states values (13, 'Idaho', 'ID');
113 | INSERT INTO states values (14, 'Illinois', 'IL');
114 | INSERT INTO states values (15, 'Indiana', 'IN');
115 | INSERT INTO states values (16, 'Iowa', 'IA');
116 | INSERT INTO states values (17, 'Kansas', 'KS');
117 | INSERT INTO states values (18, 'Kentucky', 'KY');
118 | INSERT INTO states values (19, 'Louisiana', 'LA');
119 | INSERT INTO states values (20, 'Maine', 'ME');
120 | INSERT INTO states values (21, 'Maryland', 'MD');
121 | INSERT INTO states values (22, 'Massachusetts', 'MA');
122 | INSERT INTO states values (23, 'Michigan', 'MI');
123 | INSERT INTO states values (24, 'Minnesota', 'MN');
124 | INSERT INTO states values (25, 'Mississippi', 'MS');
125 | INSERT INTO states values (26, 'Missouri', 'MO');
126 | INSERT INTO states values (27, 'Montana', 'MT');
127 | INSERT INTO states values (28, 'Nebraska', 'NE');
128 | INSERT INTO states values (29, 'Nevada', 'NV');
129 | INSERT INTO states values (30, 'New Hampshire', 'NH');
130 | INSERT INTO states values (31, 'New Jersey', 'NJ');
131 | INSERT INTO states values (32, 'New Mexico', 'NM');
132 | INSERT INTO states values (33, 'New York', 'NY');
133 | INSERT INTO states values (34, 'North Carolina', 'NC');
134 | INSERT INTO states values (35, 'North Dakota', 'ND');
135 | INSERT INTO states values (36, 'Ohio', 'OH');
136 | INSERT INTO states values (37, 'Oklahoma', 'OK');
137 | INSERT INTO states values (38, 'Oregon', 'OR');
138 | INSERT INTO states values (39, 'Pennsylvania', 'PA');
139 | INSERT INTO states values (40, 'Rhode Island', 'RI');
140 | INSERT INTO states values (41, 'South Carolina', 'SC');
141 | INSERT INTO states values (42, 'South Dakota', 'SD');
142 | INSERT INTO states values (43, 'Tennessee', 'TN');
143 | INSERT INTO states values (44, 'Texas', 'TX');
144 | INSERT INTO states values (45, 'Utah', 'UT');
145 | INSERT INTO states values (46, 'Vermont', 'VT');
146 | INSERT INTO states values (47, 'Virginia', 'VA');
147 | INSERT INTO states values (48, 'Washington', 'WA');
148 | INSERT INTO states values (49, 'West Virginia', 'WV');
149 | INSERT INTO states values (50, 'Wisconsin', 'WI');
150 | INSERT INTO states values (51, 'Wyoming', 'WY');
151 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
5 | 4.0.0
6 |
7 | com.cloudera.streaming.examples
8 | streaming-ref-app-advanced
9 | 0.1-SNAPSHOT
10 |
11 |
12 |
13 | cloudera
14 | https://repository.cloudera.com/artifactory/cloudera-repos/
15 |
16 |
17 |
18 |
19 | 2.11
20 | 1.8
21 | 3.3.2
22 | 3.7.0
23 | 2.4.0-cdh6.1.0
24 | 2.0.0-cdh6.1.0
25 | 1.8.0-cdh6.1.0
26 | 3.0.5
27 | 2.0.0
28 | 2.8.5
29 | 2.15.0
30 |
31 |
32 |
33 |
34 | org.apache.spark
35 | spark-sql-kafka-0-10_${scala.version}
36 | ${spark.version}
37 | provided
38 |
39 |
40 | org.apache.spark
41 | spark-sql_${scala.version}
42 | ${spark.version}
43 | provided
44 |
45 |
46 | com.google.code.gson
47 | gson
48 | ${gson.version}
49 |
50 |
51 | org.scalatest
52 | scalatest_${scala.version}
53 | ${scalatest.version}
54 | test
55 |
56 |
57 | org.apache.spark
58 | spark-core_${scala.version}
59 | ${spark.version}
60 | test-jar
61 | test
62 |
63 |
64 | org.mockito
65 | mockito-core
66 | ${mockito-core.version}
67 | test
68 |
69 |
70 | org.apache.spark
71 | spark-catalyst_${scala.version}
72 | ${spark.version}
73 | tests
74 | test
75 |
76 |
77 | org.apache.spark
78 | spark-sql_${scala.version}
79 | ${spark.version}
80 | tests
81 | test
82 |
83 |
84 | net.manub
85 | scalatest-embedded-kafka_${scala.version}
86 | ${scalatest-embedded-kafka.version}
87 | test
88 |
89 |
90 | org.apache.kudu
91 | kudu-spark2_${scala.version}
92 | ${kudu.version}
93 |
94 |
95 |
96 |
97 |
98 |
99 | net.alchim31.maven
100 | scala-maven-plugin
101 | ${scala.maven.plugin.version}
102 |
103 |
104 |
105 | compile
106 | testCompile
107 |
108 |
109 |
110 |
111 |
112 | maven-compiler-plugin
113 | ${maven.compiler.plugin.version}
114 |
115 | ${java.version}
116 | ${java.version}
117 |
118 |
119 |
120 | maven-assembly-plugin
121 |
122 |
123 | jar-with-dependencies
124 |
125 |
126 |
127 |
128 | make-assembly
129 | package
130 |
131 | single
132 |
133 |
134 |
135 |
136 |
137 |
138 |
139 |
140 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister:
--------------------------------------------------------------------------------
1 | com.cloudera.streaming.refapp.kudu.KuduSinkProvider
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/main/scala/com/cloudera/streaming/refapp/Application.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import java.time.Duration
7 | import java.util.concurrent.{ScheduledFuture, ScheduledThreadPoolExecutor, TimeUnit}
8 |
9 | import org.apache.spark.sql.streaming.StreamingQuery
10 | import org.apache.spark.sql.{DataFrame, SparkSession}
11 | import org.slf4j.LoggerFactory
12 |
13 | case class Sources(statesFromCluster: Source,
14 | customersFromCluster: Source,
15 | vendorsFromCluster: Source,
16 | customersFromStream: Source,
17 | vendorsFromStream: Source,
18 | transactionsFromStream: Source)
19 |
20 | case class Sinks(invalidTransactions: Sink,
21 | validTransactions: Sink,
22 | customerOrphans: Sink,
23 | vendorOrphans: Sink,
24 | customers: Sink,
25 | vendors: Sink,
26 | transactionsOperationalMetadata: Sink)
27 |
28 | class StreamingQueries(val invalidTransactions: Query,
29 | val validTransactions: Query,
30 | val customerOrphans: Query,
31 | val vendorOrphans: Query,
32 | val customers: Query,
33 | val vendors: Query,
34 | val transactionsOperationalMetadata: Query) {
35 |
36 | val allQueries = List(
37 | invalidTransactions,
38 | validTransactions,
39 | customerOrphans,
40 | vendorOrphans,
41 | customers,
42 | vendors,
43 | transactionsOperationalMetadata)
44 |
45 | def start(): Unit = {
46 | invalidTransactions.start()
47 | validTransactions.start()
48 | customerOrphans.start()
49 | vendorOrphans.start()
50 | customers.start()
51 | vendors.start()
52 | transactionsOperationalMetadata.start()
53 | }
54 | }
55 |
56 | class Application(spark: SparkSession,
57 | sources: Sources,
58 | sinks: Sinks,
59 | queryRestartDurations: Map[String, Duration] = Map.empty,
60 | cleanOutput: => Unit = noop,
61 | clusterStartup: => Unit = noop,
62 | initSources: => Unit = noop) {
63 | private val logger = LoggerFactory.getLogger(getClass)
64 |
65 | val transactionsFlow = new TransactionsFlow(
66 | spark,
67 | sources.statesFromCluster(Schemas.state),
68 | sources.customersFromCluster(Schemas.customer),
69 | sources.vendorsFromCluster(Schemas.vendor),
70 | sources.transactionsFromStream(Schemas.transaction)
71 | )
72 |
73 | val customersFlow = new CustomersFlow(spark, sources.customersFromStream(Schemas.customer))
74 | val vendorsFlow = new VendorsFlow(spark, sources.vendorsFromStream(Schemas.vendor))
75 |
76 | var streamingQueries = new StreamingQueries(
77 | // transactionsFlow.validTransactions and invalidTransactions contain columns used for internal calculations,
78 | // these do not fit to our output schemas
79 | createQuery(
80 | transactionsFlow.invalidTransactions.select("transaction_id", "customer_id", "vendor_id", "event_state", "event_timestamp", "price", "card_type"),
81 | sinks.invalidTransactions),
82 | createQuery(
83 | transactionsFlow.validTransactions.select("transaction_id", "customer_id", "vendor_id", "event_state", "event_timestamp", "price", "card_type"),
84 | sinks.validTransactions),
85 | createQuery(transactionsFlow.customerOrphans, sinks.customerOrphans),
86 | createQuery(transactionsFlow.vendorOrphans, sinks.vendorOrphans),
87 | createQuery(customersFlow.customers, sinks.customers),
88 | createQuery(vendorsFlow.vendors, sinks.vendors),
89 | createQuery(transactionsFlow.transactionsOperationalMetadata, sinks.transactionsOperationalMetadata)
90 | )
91 |
92 | def scheduleQueryRestarters(): Unit = {
93 |
94 | def restartQuery(query: Query): Unit = {
95 | println(s"Restarting query ${query.name}")
96 | try {
97 | query.restart()
98 | } catch {
99 | case e: Exception =>
100 | // log warn
101 | println(s"Could not restart query ${query.name}")
102 | e.printStackTrace()
103 | }
104 | }
105 |
106 | var schedules = List[ScheduledFuture[_]]()
107 | val executor = new ScheduledThreadPoolExecutor(1)
108 |
109 | def scheduleQueryRestarter(query:Query, period: Duration) = {
110 | println(s"Scheduling query restart of ${query.name} to $period")
111 | val task = new Runnable {
112 | def run(): Unit = restartQuery(query)
113 | }
114 | val schedule = executor.scheduleAtFixedRate(task, period.getSeconds, period.getSeconds, TimeUnit.SECONDS)
115 | schedules = schedules :+ schedule
116 | }
117 |
118 | streamingQueries.allQueries.foreach{ query =>
119 | val queryRestartPeriod = queryRestartDurations.get(query.name)
120 | queryRestartPeriod match {
121 | case Some(period) => scheduleQueryRestarter(query, period)
122 | case None => // nothing to do
123 | }
124 | }
125 |
126 | if (schedules.nonEmpty) {
127 | Runtime.getRuntime.addShutdownHook(new Thread() {
128 | override def run() {
129 | schedules.foreach { sched =>
130 | sched.cancel(true)
131 | }
132 | }
133 | })
134 | }
135 | }
136 |
137 | def start() {
138 | logger.info("Application starting")
139 | clusterStartup
140 | cleanOutput
141 | initSources
142 | streamingQueries.start()
143 | scheduleQueryRestarters()
144 | logger.info("Application started")
145 | }
146 |
147 | def createQuery(dataFrame: DataFrame, sink: Sink) = new Query {
148 | val writer = sink.createDataStreamWriter(dataFrame)
149 |
150 | var streamingQuery: Option[StreamingQuery] = None
151 |
152 | def start(): Unit =
153 | streamingQuery = Some(writer.start())
154 |
155 | def stop(): Unit =
156 | streamingQuery.foreach{q => q.stop()}
157 |
158 | def restart(): Unit = {
159 | stop()
160 | start()
161 | }
162 |
163 | def processAllAvailable(): Unit =
164 | streamingQuery.foreach{q => q.processAllAvailable() }
165 |
166 | val name = sink.name
167 | }
168 | }
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/main/scala/com/cloudera/streaming/refapp/DataFlows.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import org.apache.spark.sql.{DataFrame, SparkSession, functions}
7 |
8 | /**
9 | * Enriches incoming customer flow with state name and abbreviation.
10 | */
11 | class CustomersFlow(spark: SparkSession, customersFromStream: DataFrame) {
12 | private val customersWithWatermark = customersFromStream.withWatermark(Schemas.timestampColumnName, "10 seconds")
13 | customersWithWatermark.createOrReplaceTempView("customers_from_kafka")
14 |
15 | val customers = spark.sql(
16 | """SELECT customer_id, first_name, last_name, state_name,
17 | state_abbreviation, CAST(update_timestamp AS TIMESTAMP) update_timestamp
18 | FROM customers_from_kafka c
19 | LEFT OUTER JOIN states_from_cluster s
20 | ON c.state_id = s.state_id""")
21 | }
22 |
23 | /**
24 | * No transformation
25 | */
26 | class VendorsFlow(spark: SparkSession, vendorsFromStream: DataFrame) {
27 | private val vendorsWithWatemark = vendorsFromStream.withWatermark(Schemas.timestampColumnName, "10 seconds")
28 | vendorsWithWatemark.createOrReplaceTempView("vendors_from_kafka")
29 |
30 | val vendors = spark.sql(
31 | """SELECT vendor_id, vendor_name, phone_number,
32 | CAST(update_timestamp AS TIMESTAMP) update_timestamp
33 | FROM vendors_from_kafka""")
34 | }
35 |
36 | /**
37 | * Processes incoming transactions: validates, finds customer and vendor orphans and produces
38 | * operational metadata.
39 | */
40 | class TransactionsFlow(spark: SparkSession,
41 | statesFromCluster: DataFrame, // Schemas.states
42 | customersFromCluster: DataFrame, // Schemas.customer
43 | vendorsFromCluster: DataFrame, // Schemas.vendor
44 | transactionsFromStream: DataFrame) {
45 |
46 | import spark.implicits._
47 |
48 | private val transactionsWithWatemark = transactionsFromStream.withWatermark(Schemas.timestampColumnName, "10 seconds")
49 |
50 | statesFromCluster.createOrReplaceTempView("states_from_cluster")
51 | customersFromCluster.createOrReplaceTempView("customers_from_cluster")
52 | vendorsFromCluster.createOrReplaceTempView("vendors_from_cluster")
53 | transactionsWithWatemark.createOrReplaceTempView("transactions")
54 |
55 |
56 | // TODO consider eliminating unnecessary columns or rewrite to use pure SQL
57 | // timestamp check is not needed if the event comes from kafka
58 | private val validatedTransactions = transactionsWithWatemark.withColumn("mandatory_fields_exist", !'customer_id.isNull && !$"vendor_id".isNull &&
59 | !$"event_timestamp".isNull)
60 | .withColumn("valid_card_type", 'card_type.isin("Credit", "Debit"))
61 | .withColumn("valid_event_state", 'event_state.isin("CREATED", "SWIPED", "CANCELLED", "SIG_REQD", "AUTHORIZED", "DECLINED"))
62 | .withColumn("parsed_event_timestamp", functions.to_timestamp('event_timestamp, "yyyy-mm-dd"))
63 | .withColumn("correct_timestamp_format", !'parsed_event_timestamp.isNull)
64 | .withColumn("valid_record", 'mandatory_fields_exist && 'valid_card_type && 'valid_event_state && 'correct_timestamp_format)
65 |
66 | /**
67 | * Transaction records with missing / incorrect data
68 | */
69 | val invalidTransactions = validatedTransactions.filter(!'valid_record)
70 |
71 | /**
72 | * Transactions with complete and correct data
73 | */
74 | val validTransactions = validatedTransactions.filter('valid_record)
75 | validTransactions.createOrReplaceTempView("card_transactions_good_records")
76 |
77 | /**
78 | * Customers that did not exist in our database but were referenced in transactions
79 | */
80 | val customerOrphans = spark.sql(
81 | """SELECT customer_id, 'Unknown' first_name, 'Unknown' last_name,
82 | -1 state_id, CURRENT_TIMESTAMP() update_timestamp
83 | FROM
84 | (SELECT customer_id FROM card_transactions_good_records) sc
85 | LEFT ANTI JOIN
86 | (SELECT customer_id FROM customers_from_cluster) cc
87 | USING (customer_id)""").dropDuplicates("customer_id")
88 |
89 | customerOrphans.createOrReplaceTempView("customer_orphans")
90 |
91 | /**
92 | * Customers that did not exist in our database but were referenced in transactions
93 | */
94 | val vendorOrphans = spark.sql(
95 | """SELECT vendor_id, 'Unknown' vendor_name, 'Unknown' phone_number,
96 | CURRENT_TIMESTAMP() update_timestamp
97 | FROM
98 | (SELECT vendor_id FROM card_transactions_good_records) sv
99 | LEFT ANTI JOIN
100 | (SELECT vendor_id FROM vendors_from_cluster) cv
101 | USING (vendor_id)""").dropDuplicates("vendor_id")
102 |
103 | vendorOrphans.createOrReplaceTempView("vendor_orphans")
104 |
105 | // TODO add more operational metadata for invalid records, customer and vendor orphans
106 | /**
107 | * Operational metadata for monitoring.
108 | */
109 | val transactionsOperationalMetadata = transactionsWithWatemark
110 | .groupBy(functions.window(functions.col(Schemas.timestampColumnName), "1 minutes"))
111 | .count().as("c")
112 | .selectExpr("c.window.start as start_ts", "c.window.end as end_ts", "c.count as num_transactions")
113 |
114 | }
115 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/main/scala/com/cloudera/streaming/refapp/DataGenerator.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import java.sql.Timestamp
7 | import java.util.concurrent.{ScheduledThreadPoolExecutor, TimeUnit}
8 |
9 | import scala.collection.JavaConverters._
10 | import scala.util.Random
11 | import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
12 | import org.apache.kafka.common.serialization.{IntegerSerializer, Serializer}
13 | import org.slf4j.{Logger, LoggerFactory}
14 |
15 | object CustomerGenerator {
16 | def apply(kafkaConfig: KafkaConfig, topic: String): DataGenerator[Integer, Customer] = new CustomerGenerator(kafkaConfig, topic).generator
17 | }
18 |
19 | object VendorGenerator {
20 | def apply(kafkaConfig: KafkaConfig, topic: String): DataGenerator[Integer, Vendor] = new VendorGenerator(kafkaConfig, topic).generator
21 | }
22 |
23 | object TransactionGenerator {
24 | def apply(kafkaConfig: KafkaConfig, topic: String, recordsPerSec: Int = 1): DataGenerator[Integer, Transaction] = new TransactionGenerator(kafkaConfig, topic, recordsPerSec).generator
25 | }
26 |
27 | class CustomerGenerator(kafkaConfig: KafkaConfig, topic: String) {
28 | val generator = new DataGenerator[Integer, Customer](
29 | kafkaConfig,
30 | topic = topic,
31 | createValue = Customer(
32 | customer_id = Random.nextInt(10) + 1,
33 | state_id = if (Random.nextInt(100) < 5) -1 else Random.nextInt(51) + 1,
34 | first_name = randomFirstName,
35 | last_name = randomLastName,
36 | update_timestamp = new Timestamp(System.currentTimeMillis() - Random.nextInt(48 * 60 * 60 * 1000))),
37 | getKey = _.customer_id,
38 | keySerializer = new IntegerSerializer(),
39 | valueSerializer = new CustomerSerializer())
40 |
41 | private val firstNames = Seq("John", "Jane", "Alex", "Jessica")
42 |
43 | private def randomFirstName = firstNames(Random.nextInt(firstNames.length))
44 |
45 | private val lastNames = Seq("Doe", "Smith", "Tailor", "Hamilton")
46 |
47 | private def randomLastName = lastNames(Random.nextInt(lastNames.length))
48 | }
49 |
50 | class VendorGenerator(kafkaConfig: KafkaConfig, topic: String) {
51 | val generator = new DataGenerator[Integer, Vendor](
52 | kafkaConfig,
53 | topic = topic,
54 | createValue = Vendor(
55 | vendor_id = Random.nextInt(10) + 1,
56 | vendor_name = randomVendorName,
57 | phone_number = randomPhoneNumber,
58 | update_timestamp = new Timestamp(System.currentTimeMillis() - Random.nextInt(48 * 60 * 60 * 1000))),
59 | getKey = _ => null,
60 | keySerializer = new IntegerSerializer(),
61 | valueSerializer = new VendorSerializer())
62 |
63 | private val vendorNames = Seq("Acme Corp.", "Cyberdyne Systems", "Hooli", "Initech", "Stark Industries", "Wayne Enterprises")
64 |
65 | private def randomVendorName = vendorNames(Random.nextInt(vendorNames.length))
66 |
67 | private def randomPhoneNumber = f"+1-${Random.nextInt(1000)}%03d-555-${Random.nextInt(10000)}%04d"
68 | }
69 |
70 | class TransactionGenerator(kafkaConfig: KafkaConfig, topic: String, recordsPerSecond: Int) {
71 | val generator = new DataGenerator[Integer, Transaction](
72 | kafkaConfig,
73 | topic = topic,
74 | createValue = Transaction(
75 | transaction_id = Random.alphanumeric.take(3).mkString,
76 | customer_id = if (Random.nextInt(100) < 20) None else Some(Random.nextInt(10) + 1),
77 | vendor_id = if (Random.nextInt(100) < 20) None else Some(Random.nextInt(10) + 1),
78 | event_state = randomEventState,
79 | event_timestamp = new Timestamp(System.currentTimeMillis() - Random.nextInt(60 * 1000)),
80 | price = if (Random.nextInt(100) < 20) None else Some(Random.nextInt(100000).toString),
81 | card_type = randomCardType),
82 | getKey = transaction => {
83 | val id: Integer = if (transaction.customer_id.isDefined) transaction.customer_id.get else null
84 | id
85 | },
86 | new IntegerSerializer(),
87 | new TransactionSerializer(),
88 | recordsPerSecond
89 | )
90 |
91 | private val states = Seq("CREATED", "SWIPED", "AUTHORIZED", "INVALID")
92 |
93 | private def randomEventState = if (Random.nextInt(100) < 20) None else Some(states(Random.nextInt(states.length)))
94 |
95 | private val cardTypes = Seq("Credit", "Debit", "Whatever")
96 |
97 | private def randomCardType = if (Random.nextInt(100) < 20) None else Some(cardTypes(Random.nextInt(cardTypes.length)))
98 | }
99 |
100 | class DataGenerator[K, V](kafkaConfig: KafkaConfig,
101 | topic: String,
102 | createValue: => V,
103 | getKey: V => K,
104 | keySerializer: Serializer[K],
105 | valueSerializer: Serializer[V],
106 | recordsPerSecond: Int = 1) {
107 |
108 | val logger : Logger = LoggerFactory.getLogger(getClass)
109 |
110 | def start(): Unit = {
111 | logger.info("Data generator starting")
112 | val config: Map[String, Object] = kafkaConfig.kafkaParams
113 | val producer = new KafkaProducer(config.asJava, keySerializer, valueSerializer)
114 |
115 | def generate(recordCount : Int) = try {
116 | for (_ <- 1 to recordCount) {
117 | val value: V = createValue
118 | val key: K = getKey(value)
119 | logger.debug(s"Producing to $topic: $value")
120 | producer.send(new ProducerRecord(topic, key, value))
121 | }
122 | } catch {
123 | case e: Exception =>
124 | logger.error("Exception while producing", e)
125 | System.exit(1)
126 | }
127 |
128 | val ex = new ScheduledThreadPoolExecutor(1)
129 | val task = new Runnable {
130 | def run(): Unit = generate(recordsPerSecond)
131 | }
132 | val sched = ex.scheduleAtFixedRate(task, 1, 1, TimeUnit.SECONDS)
133 |
134 | Runtime.getRuntime.addShutdownHook(new Thread() {
135 | override def run() {
136 | logger.info("Data generator stopping")
137 | sched.cancel(false)
138 | producer.close()
139 | logger.info("Data generator stopped")
140 | }
141 | })
142 | logger.info("Data generator started")
143 | }
144 | }
145 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/main/scala/com/cloudera/streaming/refapp/DeployedDataGenerator.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | /**
7 | * Created by andrasbeni on 12/1/18.
8 | */
9 | object DeployedDataGenerator {
10 |
11 | def main(args: Array[String]) {
12 |
13 | if (args.length != 2) {
14 | sys.error(
15 | """Usage:
16 | |com.cloudera.streaming.refapp.DeployedDataGenerator producer.config
17 | |producer.config path to kafka client properties
18 | |transactions.per.sec number of records produced to "transaction" topic per second
19 | """.stripMargin)
20 | }
21 | val Array(producerConfig, recordsPerSec) = args
22 | val kafkaConfig: KafkaConfig = KafkaConfig.fromPropertiesFile(producerConfig)
23 |
24 | CustomerGenerator(kafkaConfig, "customer").start()
25 | VendorGenerator(kafkaConfig, "vendor").start()
26 | TransactionGenerator(kafkaConfig, "transaction", recordsPerSec.toInt).start()
27 |
28 | }
29 |
30 | }
31 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/main/scala/com/cloudera/streaming/refapp/DeployedStructuredStreams.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import org.apache.spark.sql.SparkSession
7 | import org.slf4j.LoggerFactory
8 |
9 | /**
10 | * A long running streaming application that can be submitted to a running Spark service.
11 | * It reads static data from Kudu, streaming data from Kafka and writes output to Kudu
12 | */
13 | object DeployedStructuredStreams {
14 |
15 | def main(args: Array[String]) {
16 |
17 | val logger = LoggerFactory.getLogger(getClass)
18 |
19 | if (args.length < 2) {
20 | sys.error(
21 | """Usage:
22 | |com.cloudera.streaming.refapp.DeployedStructuredStreams consumer.config kudu-master timeToLive
23 | |consumer.config path to kafka client properties
24 | |kudu-master host:port pair pointing to a kudu master instance
25 | |timeToLive optional, if specified the application will be stopped after timeToLive seconds, useful for testing
26 | """.stripMargin)
27 | }
28 | // extract first two arguments
29 | val Array(consumerConfig, kuduMaster, _*) = args
30 | // read optional argument
31 | val timeToLive = if (args.length > 2) Some(args(2).toInt) else None
32 |
33 | val spark = SparkSession.builder().appName("streaming-ref").getOrCreate()
34 |
35 | val kafkaConfig: KafkaConfig = KafkaConfig.fromPropertiesFile(consumerConfig)
36 | val kafkaSource = new KafkaSource(spark, kafkaConfig)
37 |
38 | val kuduDatabase = "streaming_ref"
39 | val kuduSource = new KuduSource(spark, kuduMaster, kuduDatabase)
40 | val kuduSink = new KuduSink(kuduMaster, kuduDatabase, defaultCheckpointLocation)
41 |
42 | val application = new Application(
43 | spark,
44 | Sources(
45 | statesFromCluster = kuduSource.loadTable("states"),
46 | customersFromCluster = kuduSource.loadTable("customers"),
47 | vendorsFromCluster = kuduSource.loadTable("vendors"),
48 | customersFromStream = kafkaSource.jsonStreamWithKafkaTimestamp("customer"),
49 | vendorsFromStream = kafkaSource.jsonStreamWithTimestampFromMessage("vendor", "update_timestamp"),
50 | transactionsFromStream = kafkaSource.jsonStreamWithTimestampFromMessage("transaction", "event_timestamp")
51 | ),
52 | Sinks(
53 | validTransactions = kuduSink.writeTable("valid_transactions"),
54 | invalidTransactions = kuduSink.writeTable("invalid_transactions"),
55 | customerOrphans = kuduSink.writeTable("customer_orphans"),
56 | vendorOrphans = kuduSink.writeTable("vendor_orphans"),
57 | customers = kuduSink.writeTable("customers"),
58 | vendors = kuduSink.writeTable("vendors"),
59 | transactionsOperationalMetadata = kuduSink.writeTable("transactions_operational_metadata")
60 | ))
61 |
62 | application.start()
63 |
64 | timeToLive match {
65 | case Some(tl) =>
66 | logger.info(s"Running application for $tl seconds")
67 | Thread.sleep(tl * 1000)
68 | logger.info("Stopping application")
69 | case None => spark.streams.awaitAnyTermination()
70 | }
71 | }
72 | }
73 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/main/scala/com/cloudera/streaming/refapp/Kafka.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import java.io.{FileInputStream, InputStream}
7 | import java.util.Properties
8 |
9 | import scala.collection.JavaConverters._
10 |
11 | import org.apache.spark.sql.types.StructType
12 | import org.apache.spark.sql.{DataFrame, SparkSession, functions}
13 |
14 | /**
15 | * Reads/builds configuration for kafka clients that are either standalone or live in Spark.
16 | */
17 | object KafkaConfig {
18 |
19 | /**
20 | * Reads configuration from property file
21 | */
22 | def fromPropertiesFile(configFile : String) = {
23 | var inputStream : Option[InputStream] = None
24 | val kafkaParams = try {
25 | inputStream = Some(new FileInputStream(configFile))
26 | val params = new Properties()
27 | params.load(inputStream.get)
28 | inputStream.get.close()
29 | params.asScala.toMap
30 | } finally {
31 | inputStream.foreach(_.close())
32 | }
33 | new KafkaConfig(kafkaParams)
34 | }
35 |
36 | /**
37 | * Builds configuration for brokers using PLAINTEXT protocol.
38 | */
39 | def fromBrokerList(bootstrapServers: String) = {
40 |
41 | new KafkaConfig(Map[String, String](
42 | "bootstrap.servers" -> bootstrapServers,
43 | "security.protocol" -> "PLAINTEXT"))
44 | }
45 | }
46 |
47 | class KafkaConfig(val kafkaParams : Map[String, String]) {
48 |
49 | /**
50 | * Converts plain kafka configuration for usage in Spark.
51 | */
52 | val kafkaParamsForSpark: Map[String, String] = kafkaParams.map {
53 | case (key, value) => "kafka." + key -> value
54 | }
55 | }
56 | /**
57 | * Creates streaming Sources reading Kafka topics.
58 | */
59 | class KafkaSource(spark: SparkSession, kafkaConfig: KafkaConfig) {
60 |
61 | private def loadStream(topic:String, startingOffset: String) = {
62 |
63 | val params = kafkaConfig.kafkaParamsForSpark +
64 | ("subscribe" -> topic) +
65 | ("startingoffsets" -> startingOffset)
66 | spark.readStream.format("kafka").options(params).load()
67 | }
68 |
69 | /**
70 | * Creates a dataframe from a kafka topic containing Strings.
71 | * Useful for testing and debugging.
72 | */
73 | def stringStream(topic: String): DataFrame = loadStream(topic, "earliest").selectExpr("CAST(value AS STRING)")
74 |
75 | /**
76 | * Creates a streaming source that reads JSON records.
77 | * The DataFrame will include the timestamp that kafka added to the message, called Schemas.timestampColumnName.
78 | */
79 | def jsonStreamWithKafkaTimestamp(topic: String)(schema: StructType): DataFrame = {
80 |
81 | import spark.implicits._
82 |
83 | loadStream(topic, "latest")
84 | .withColumn(Schemas.timestampColumnName, functions.col("timestamp"))
85 | .selectExpr("CAST(value AS STRING)", Schemas.timestampColumnName)
86 | .select(functions.from_json('value, schema) as "entity", functions.col(Schemas.timestampColumnName))
87 | .select("entity.*", Schemas.timestampColumnName)
88 | }
89 |
90 | /**
91 | * Creates a streaming source that reads JSON records.
92 | * The DataFrame will include the timestamp from the original message, called Schemas.timestampColumnName.
93 | */
94 | def jsonStreamWithTimestampFromMessage(topic: String, timestampColumnName: String)(schema: StructType): DataFrame = {
95 |
96 | import spark.implicits._
97 |
98 | // TODO simplify selects
99 | loadStream(topic, "latest")
100 | .selectExpr("CAST(value AS STRING)")
101 | .select(functions.from_json('value, schema) as "entity")
102 | .selectExpr("entity.*", s"entity.$timestampColumnName as ${Schemas.timestampColumnName}")
103 | }
104 | }
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/main/scala/com/cloudera/streaming/refapp/Kudu.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import org.apache.kudu.spark.kudu._
7 | import org.apache.spark.sql.streaming.DataStreamWriter
8 | import org.apache.spark.sql.types.StructType
9 | import org.apache.spark.sql.{DataFrame, Row, SparkSession}
10 |
11 | /**
12 | * Creates static Sources reading Kudu tables.
13 | */
14 | class KuduSource(spark: SparkSession, master: String, database: String) {
15 |
16 | /**
17 | * Creates a Source that reads content from a table.
18 | * It will register a TempView with the same name in Spark session, so SQL queries can use it.
19 | */
20 | def loadTable(name: String)(ignored: StructType) = {
21 | val fullTableName = s"impala::$database.$name"
22 | val df = spark
23 | .read
24 | .options(
25 | Map(
26 | "kudu.master" -> master,
27 | "kudu.table" -> fullTableName)).kudu
28 | df.createOrReplaceTempView(name)
29 | df
30 | }
31 | }
32 |
33 | /**
34 | * Creates Sinks that produce streaming output to Kudu tables.
35 | *
36 | * @param checkpointLocation provides the path where the checkpoints are stored, given the name of the Sink
37 | */
38 | class KuduSink(master: String, database: String, checkpointLocation: String => String) {
39 |
40 | def writeTable(sinkName: String, triggerSeconds: Int = 10) =
41 | new Sink {
42 | override def createDataStreamWriter(df: DataFrame): DataStreamWriter[Row] = {
43 | val fullTableName = s"impala::$database.$name"
44 | df
45 | .writeStream
46 | .format("kudu")
47 | .option("kudu.master", master)
48 | .option("kudu.table", fullTableName)
49 | .option("checkpointLocation", checkpointLocation(name))
50 | .option("retries", "3")
51 | .outputMode("update")
52 | }
53 |
54 | override val name: String = sinkName
55 | }
56 |
57 | }
58 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/main/scala/com/cloudera/streaming/refapp/Schemas.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import java.lang.reflect.Type
7 | import java.util
8 |
9 | import com.google.gson._
10 | import org.apache.kafka.common.serialization.Serializer
11 |
12 | import org.apache.spark.sql.Encoders
13 | import org.apache.spark.sql.types.{StringType, StructType}
14 |
15 | /**
16 | * Contains schemas and serializers for various domain objects.
17 | */
18 | object Schemas {
19 | val timestampColumnName = "timestamp"
20 | val transaction: StructType = Encoders.product[Transaction].schema
21 | val vendor: StructType = Encoders.product[Vendor].schema
22 | val customer: StructType = Encoders.product[Customer].schema
23 |
24 | val state: StructType = Encoders.product[State].schema
25 | val plainText: StructType = new StructType().
26 | add("value", StringType)
27 | }
28 |
29 | class OptionSerializer extends JsonSerializer[Option[Any]] {
30 | override def serialize(src: Option[Any], typeOfSrc: Type, context: JsonSerializationContext): JsonElement = {
31 | src match {
32 | case None => JsonNull.INSTANCE
33 | case Some(v) => context.serialize(v)
34 | }
35 | }
36 | }
37 |
38 | class TransactionSerializer extends Serializer[Transaction] {
39 |
40 | private val gson = new GsonBuilder().setDateFormat("yyyy-MM-dd HH:mm:ss").registerTypeHierarchyAdapter(classOf[Option[Any]], new OptionSerializer).create()
41 |
42 | override def configure(map: util.Map[String, _], b: Boolean): Unit = {}
43 |
44 | override def serialize(topic: String, transaction: Transaction): Array[Byte] = {
45 | gson.toJson(transaction).getBytes
46 | }
47 |
48 | override def close(): Unit = {}
49 | }
50 |
51 | class CustomerSerializer extends Serializer[Customer] {
52 |
53 | private val gson = new GsonBuilder().setDateFormat("yyyy-MM-dd").create()
54 |
55 | override def configure(map: util.Map[String, _], b: Boolean): Unit = {}
56 |
57 | override def serialize(topic: String, customer: Customer): Array[Byte] = {
58 | gson.toJson(customer).getBytes
59 | }
60 |
61 | override def close(): Unit = {}
62 | }
63 |
64 | class VendorSerializer extends Serializer[Vendor] {
65 |
66 | private val gson = new GsonBuilder().setDateFormat("yyyy-MM-dd").create()
67 |
68 | override def configure(map: util.Map[String, _], b: Boolean): Unit = {}
69 |
70 | override def serialize(topic: String, vendor: Vendor): Array[Byte] = {
71 | gson.toJson(vendor).getBytes
72 | }
73 |
74 | override def close(): Unit = {}
75 | }
76 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/main/scala/com/cloudera/streaming/refapp/kudu/KuduSink.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Modifications copyright (C) 2019 Cloudera Inc
3 | */
4 | package com.cloudera.streaming.refapp.kudu
5 |
6 | import org.apache.kudu.spark.kudu.KuduContext
7 | import org.apache.spark.sql.execution.streaming.Sink
8 | import org.apache.spark.sql.{DataFrame, SQLContext}
9 | import org.slf4j.LoggerFactory
10 |
11 | import scala.util.control.NonFatal
12 |
13 | object KuduSink {
14 | def withDefaultContext(sqlContext: SQLContext, parameters: Map[String, String]) =
15 | new KuduSink(new KuduContext(parameters("kudu.master"), sqlContext.sparkContext), parameters)
16 | }
17 |
18 | /**
19 | * A simple Structured Streaming sink which writes the data frame to Kudu.
20 | * It preserves exactly once semantics, as it's idempotent in the face of
21 | * multiple attempts to add the same batch.
22 | *
23 | * It uses the following parameters:
24 | * kudu.master - host:port pair of a kudu master node
25 | * kudu.table - full table name
26 | * checkpointLocation - where the checkpoint will be stored
27 | */
28 | class KuduSink(initKuduContext: => KuduContext, parameters: Map[String, String]) extends Sink {
29 |
30 | private val logger = LoggerFactory.getLogger(getClass)
31 |
32 | private var kuduContext = initKuduContext
33 |
34 | private val tablename = parameters("kudu.table")
35 |
36 | private val retries = parameters.getOrElse("retries", "1").toInt
37 | require(retries >= 0, "retries must be non-negative")
38 |
39 | logger.info(s"Created Kudu sink writing to table $tablename")
40 |
41 | override def addBatch(batchId: Long, data: DataFrame): Unit = {
42 | for (attempt <- 0 to retries) {
43 | try {
44 | kuduContext.upsertRows(data, tablename)
45 | return
46 | } catch {
47 | case NonFatal(e) =>
48 | if (attempt < retries) {
49 | logger.warn("Kudu upsert error, retrying...", e)
50 | kuduContext = initKuduContext
51 | }
52 | else {
53 | logger.error("Kudu upsert error, exhausted", e)
54 | throw e
55 | }
56 | }
57 | }
58 | }
59 | }
60 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/main/scala/com/cloudera/streaming/refapp/kudu/KuduSinkProvider.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Modifications copyright (C) 2019 Cloudera Inc
3 | */
4 | package com.cloudera.streaming.refapp.kudu
5 |
6 | import org.apache.spark.sql.SQLContext
7 | import org.apache.spark.sql.execution.streaming.Sink
8 | import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider}
9 | import org.apache.spark.sql.streaming.OutputMode
10 |
11 | /**
12 | * Registers KuduSink to Spark streaming, so it can be used with format "kudu".
13 | *
14 | * Note: to make it effective you need META-INF/services/org.apache.spark.sql.sources.DataSourceRegister to
15 | * refer to this class.
16 | */
17 | class KuduSinkProvider extends StreamSinkProvider with DataSourceRegister {
18 |
19 | override def createSink(sqlContext: SQLContext,
20 | parameters: Map[String, String],
21 | partitionColumns: Seq[String],
22 | outputMode: OutputMode): Sink = {
23 | require(outputMode == OutputMode.Update, "only 'update' OutputMode is supported")
24 | KuduSink.withDefaultContext(sqlContext, parameters)
25 | }
26 |
27 | override def shortName(): String = "kudu"
28 | }
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/main/scala/com/cloudera/streaming/refapp/package.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming
5 |
6 | import java.sql.Timestamp
7 | import java.util.UUID
8 |
9 | import org.apache.spark.sql.streaming.DataStreamWriter
10 | import org.apache.spark.sql.types.StructType
11 | import org.apache.spark.sql.{DataFrame, Row}
12 | import org.slf4j.LoggerFactory
13 |
14 | package object refapp {
15 |
16 | private val logger = LoggerFactory.getLogger(getClass)
17 |
18 | val noop = {}
19 |
20 | /**
21 | * A Source creates a static or streaming DataFrame. Incoming data is parsed using the given schema.
22 | */
23 | type Source = StructType => DataFrame
24 |
25 | /**
26 | * Represents a streaming query that is executing continuously in the background as new data arrives
27 | */
28 | trait Query {
29 | def start(): Unit
30 | def stop(): Unit
31 | def restart(): Unit
32 | def processAllAvailable(): Unit
33 | val name: String
34 | }
35 |
36 | /**
37 | * Connects the output of a streaming query to a storage or messaging system.
38 | */
39 | trait Sink {
40 | val name: String
41 | def createDataStreamWriter(df: DataFrame): DataStreamWriter[Row]
42 | }
43 |
44 | // domain objects
45 | case class Transaction(transaction_id: String,
46 | customer_id: Option[Int],
47 | vendor_id: Option[Int],
48 | event_state: Option[String],
49 | event_timestamp: Timestamp,
50 | price: Option[String],
51 | card_type: Option[String])
52 |
53 | case class Vendor(vendor_id: Int,
54 | vendor_name: String,
55 | phone_number: String,
56 | update_timestamp: Timestamp)
57 |
58 | case class Customer(customer_id: Int,
59 | state_id: Int,
60 | first_name: String,
61 | last_name: String,
62 | update_timestamp: Timestamp)
63 |
64 | case class State(state_id: Int,
65 | state_name: String,
66 | state_abbreviation: String)
67 |
68 |
69 |
70 | private val baseCheckpointLocation = "/tmp/temporary-" + UUID.randomUUID.toString
71 | logger.info(s"Storing Spark checkpoints in $baseCheckpointLocation")
72 |
73 | def defaultCheckpointLocation(streamName: String) = s"$baseCheckpointLocation/$streamName"
74 |
75 | }
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/resources/samples/cluster/customers/customers.json:
--------------------------------------------------------------------------------
1 | {"customer_id": 1, "first_name": "John", "last_name": "Doe", "state_abbreviation": "AL", "state_name": "Alabama", "update_timestamp": "2018-01-01"},
2 | {"customer_id": 2, "first_name": "Jane", "last_name": "Miller", "state_abbreviation": "AK", "state_name": "Alaska", "update_timestamp": "2018-02-02"}
3 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/resources/samples/cluster/states/states.json:
--------------------------------------------------------------------------------
1 | {"state_id": -1, "state_name": "Unknown", "state_abbreviation": "??"},
2 | {"state_id": 1, "state_name": "Alabama", "state_abbreviation": "AL"},
3 | {"state_id": 2, "state_name": "Alaska", "state_abbreviation": "AK"},
4 | {"state_id": 3, "state_name": "Arizona", "state_abbreviation": "AZ"},
5 | {"state_id": 4, "state_name": "Arkansas", "state_abbreviation": "AR"},
6 | {"state_id": 5, "state_name": "California", "state_abbreviation": "CA"},
7 | {"state_id": 6, "state_name": "Colorado", "state_abbreviation": "CO"},
8 | {"state_id": 7, "state_name": "Connecticut", "state_abbreviation": "CT"},
9 | {"state_id": 8, "state_name": "Delaware", "state_abbreviation": "DE"},
10 | {"state_id": 9, "state_name": "District of Columbia", "state_abbreviation": "DC"},
11 | {"state_id": 10, "state_name": "Florida", "state_abbreviation": "FL"},
12 | {"state_id": 11, "state_name": "Georgia", "state_abbreviation": "GA"},
13 | {"state_id": 12, "state_name": "Hawaii", "state_abbreviation": "HI"},
14 | {"state_id": 13, "state_name": "Idaho", "state_abbreviation": "ID"},
15 | {"state_id": 14, "state_name": "Illinois", "state_abbreviation": "IL"},
16 | {"state_id": 15, "state_name": "Indiana", "state_abbreviation": "IN"},
17 | {"state_id": 16, "state_name": "Iowa", "state_abbreviation": "IA"},
18 | {"state_id": 17, "state_name": "Kansas", "state_abbreviation": "KS"},
19 | {"state_id": 18, "state_name": "Kentucky", "state_abbreviation": "KY"},
20 | {"state_id": 19, "state_name": "Louisiana", "state_abbreviation": "LA"},
21 | {"state_id": 20, "state_name": "Maine", "state_abbreviation": "ME"},
22 | {"state_id": 21, "state_name": "Maryland", "state_abbreviation": "MD"},
23 | {"state_id": 22, "state_name": "Massachusetts", "state_abbreviation": "MA"},
24 | {"state_id": 23, "state_name": "Michigan", "state_abbreviation": "MI"},
25 | {"state_id": 24, "state_name": "Minnesota", "state_abbreviation": "MN"},
26 | {"state_id": 25, "state_name": "Mississippi", "state_abbreviation": "MS"},
27 | {"state_id": 26, "state_name": "Missouri", "state_abbreviation": "MO"},
28 | {"state_id": 27, "state_name": "Montana", "state_abbreviation": "MT"},
29 | {"state_id": 28, "state_name": "Nebraska", "state_abbreviation": "NE"},
30 | {"state_id": 29, "state_name": "Nevada", "state_abbreviation": "NV"},
31 | {"state_id": 30, "state_name": "New Hampshire", "state_abbreviation": "NH"},
32 | {"state_id": 31, "state_name": "New Jersey", "state_abbreviation": "NJ"},
33 | {"state_id": 32, "state_name": "New Mexico", "state_abbreviation": "NM"},
34 | {"state_id": 33, "state_name": "New York", "state_abbreviation": "NY"},
35 | {"state_id": 34, "state_name": "North Carolina", "state_abbreviation": "NC"},
36 | {"state_id": 35, "state_name": "North Dakota", "state_abbreviation": "ND"},
37 | {"state_id": 36, "state_name": "Ohio", "state_abbreviation": "OH"},
38 | {"state_id": 37, "state_name": "Oklahoma", "state_abbreviation": "OK"},
39 | {"state_id": 38, "state_name": "Oregon", "state_abbreviation": "OR"},
40 | {"state_id": 39, "state_name": "Pennsylvania", "state_abbreviation": "PA"},
41 | {"state_id": 40, "state_name": "Rhode Island", "state_abbreviation": "RI"},
42 | {"state_id": 41, "state_name": "South Carolina", "state_abbreviation": "SC"},
43 | {"state_id": 42, "state_name": "South Dakota", "state_abbreviation": "SD"},
44 | {"state_id": 43, "state_name": "Tennessee", "state_abbreviation": "TN"},
45 | {"state_id": 44, "state_name": "Texas", "state_abbreviation": "TX"},
46 | {"state_id": 45, "state_name": "Utah", "state_abbreviation": "UT"},
47 | {"state_id": 46, "state_name": "Vermont", "state_abbreviation": "VT"},
48 | {"state_id": 47, "state_name": "Virginia", "state_abbreviation": "VA"},
49 | {"state_id": 48, "state_name": "Washington", "state_abbreviation": "WA"},
50 | {"state_id": 49, "state_name": "West Virginia", "state_abbreviation": "WV"},
51 | {"state_id": 50, "state_name": "Wisconsin", "state_abbreviation": "WI"},
52 | {"state_id": 51, "state_name": "Wyoming", "state_abbreviation": "WY"}
53 |
54 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/resources/samples/cluster/vendors/vendors.json:
--------------------------------------------------------------------------------
1 | {"vendor_id": 1, "vendor_name": "Apple", "phone_number": "123456", "update_timestamp": "2018-11-13"}
2 | {"vendor_id": 2, "vendor_name": "Dell", "phone_number": "345678", "update_timestamp": "2018-11-13"}
3 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/resources/samples/kafka/customers/customers.json:
--------------------------------------------------------------------------------
1 | {"customer_id": 1, "first_name": "John", "last_name": "Doe", "state_id": 3, "update_timestamp": "2018-11-13"},
2 | {"customer_id": 2, "first_name": "Jane", "last_name": "Miller", "state_id": 5, "update_timestamp": "2018-11-13"}
3 | {"customer_id": 3, "first_name": "Joe", "last_name": "Smith", "state_id": 2, "update_timestamp": "2018-11-13"}
4 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/resources/samples/kafka/transactions/transactions-1.json:
--------------------------------------------------------------------------------
1 | {"transaction_id": "11", "customer_id": 1, "vendor_id": 1, "event_state": "CREATED", "event_timestamp": "2018-11-12 09:42:00", "price": "100", "card_type": "Credit"}
2 | {"transaction_id": "12", "customer_id": 2, "vendor_id": 2, "event_state": "SWIPED", "event_timestamp": "2018-11-13 09:43:00", "price": "100", "card_type": "Debit"}
3 | {"transaction_id": "13", "customer_id": 1, "vendor_id": 1, "event_state": "SWIPED", "event_timestamp": "2018-11-13 09:43:01", "price": "100", "card_type": "Debit"}
4 | {"transaction_id": "13", "customer_id": 2, "vendor_id": 2, "event_state": "AUTHORIZED", "event_timestamp": "2018-11-13 09:44:01", "price": "100", "card_type": "Debit"}
5 | {"transaction_id": "13", "customer_id": 1, "vendor_id": 1, "event_state": "AUTHORIZED", "event_timestamp": "2018-11-13 09:45:05", "price": "100", "card_type": "Debit"}
6 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/resources/samples/kafka/transactions/transactions-2.json:
--------------------------------------------------------------------------------
1 | {"transaction_id": "21", "customer_id": 100, "vendor_id": 2, "event_state": "SWIPED", "event_timestamp": "2018-11-13 09:45:01", "price": "100", "card_type": "Debit"}
2 | {"transaction_id": "22", "customer_id": 1, "vendor_id": 200, "event_state": "SWIPED", "event_timestamp": "2018-11-13 09:45:02", "price": "100", "card_type": "Debit"}
3 | {"transaction_id": "23", "customer_id": 1, "vendor_id": 1, "event_state": "INVALID", "event_timestamp": "2018-11-13 09:45:05", "price": "100", "card_type": "Credit"}
4 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/resources/samples/kafka/transactions/transactions-3.json:
--------------------------------------------------------------------------------
1 | {"transaction_id": "31", "customer_id": 1, "vendor_id": 1, "event_state": "CREATED", "event_timestamp": "2018-11-13 09:45:00", "price": "100", "card_type": "Whatever"}
2 | {"transaction_id": "32", "customer_id": 1, "vendor_id": 1, "event_state": "CREATED", "event_timestamp": "2018-11-14 09:47:00", "price": "100", "card_type": "Credit"}
3 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/resources/samples/kafka/transactions/transactions-4.json:
--------------------------------------------------------------------------------
1 | {"transaction_id": "41", "customer_id": 2, "vendor_id": 1, "event_state": "CREATED", "event_timestamp": "2018-11-14 10:47:00", "price": "100", "card_type": "Credit"}
2 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/resources/samples/kafka/vendors/vendors.json:
--------------------------------------------------------------------------------
1 | {"vendor_id": 1, "vendor_name": "Apple", "phone_number": "111111", "update_timestamp": "2018-11-13"}
2 | {"vendor_id": 2, "vendor_name": "Dell", "phone_number": "222222", "update_timestamp": "2018-11-13"}
3 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/scala/com/cloudera/streaming/refapp/EmbeddedKafka.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import net.manub.embeddedkafka.{EmbeddedKafka, EmbeddedKafkaConfig}
7 |
8 | object EmbeddedKafkaBroker {
9 |
10 | def embeddedKafkaConfig = EmbeddedKafkaConfig.defaultConfig
11 |
12 | def defaultKafkaConfig = KafkaConfig.fromBrokerList(
13 | s"localhost:${embeddedKafkaConfig.kafkaPort}")
14 |
15 | def start() {
16 | Runtime.getRuntime.addShutdownHook(new Thread() {
17 | override def run() {
18 | EmbeddedKafkaBroker.stop()
19 | }
20 | })
21 |
22 | EmbeddedKafka.start()
23 | }
24 |
25 | def stop() { EmbeddedKafka.stop()}
26 |
27 | def publishStringMessageToKafka(topic: String, message: String) { EmbeddedKafka.publishStringMessageToKafka(topic, message) }
28 | }
29 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/scala/com/cloudera/streaming/refapp/EmbeddedSpark.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import java.util.UUID
7 |
8 | import org.apache.spark.SparkConf
9 | import org.apache.spark.sql.SparkSession
10 |
11 | object EmbeddedSpark {
12 |
13 | val sparkSession: SparkSession = SparkSession.
14 | builder()
15 | .config(
16 | new SparkConf()
17 | .setMaster("local[*]")
18 | .setAppName("test")
19 | .set("spark.ui.enabled", "false")
20 | .set("spark.sql.shuffle.partitions", "1")
21 | .set("spark.app.id", UUID.randomUUID.toString))
22 | .getOrCreate()
23 |
24 | }
25 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/scala/com/cloudera/streaming/refapp/Files.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import java.io.File
7 |
8 | import org.apache.commons.io.FileUtils
9 |
10 | import org.apache.spark.sql.streaming.{DataStreamWriter, OutputMode, Trigger}
11 | import org.apache.spark.sql.types.StructType
12 | import org.apache.spark.sql.{DataFrame, Row, SparkSession, functions}
13 | import org.slf4j.LoggerFactory
14 |
15 |
16 | /**
17 | * Creates static and streaming Sources reading local or HDFS files.
18 | * Used for testing.
19 | */
20 | class FileSources(spark: SparkSession, inputDir: String) {
21 |
22 | /**
23 | * Creates a streaming source that reads JSON files.
24 | * It assumes that the files are located in the inputDir/kafka/fileName directory
25 | */
26 | def jsonStream(fileName: String, timestampColumnName: String)(schema: StructType): DataFrame =
27 | spark.readStream
28 | .format("json")
29 | .option("maxFilesPerTrigger", "1") // ensures that we have multiple minibatches: if we have more files each minibatch reads only one of them
30 | .schema(schema)
31 | .load(s"$inputDir/kafka/${fileName}")
32 | .repartition(1)
33 | .withColumn(Schemas.timestampColumnName, functions.col(timestampColumnName))
34 |
35 | import spark.implicits._
36 |
37 | /**
38 | * Creates a static source that reads JSON files.
39 | * It assumes that the files are located in the inputDir/cluster/fileName directory
40 | */
41 | def jsonFile(fileName: String)(schema: StructType): DataFrame =
42 | spark.sparkContext.textFile(s"$inputDir/cluster/${fileName}", 1).toDF.repartition(1)
43 | .select(functions.from_json('value, schema) as 'entity).select("entity.*")
44 | }
45 |
46 | /**
47 | * Creates Sinks that produce streaming output to CSV files.
48 | * Local or HDFS directory where the CSV files are written.
49 | *
50 | * @param outputDir
51 | * @param checkpointLocation provides the path where the checkpoints are stored, given the name of the Sink
52 | */
53 | class FileSinks(outputDir: String, checkpointLocation: String => String) {
54 |
55 | private val logger = LoggerFactory.getLogger(getClass)
56 |
57 | def csv(sinkName: String, triggerSeconds: Int = 10) =
58 | new Sink {
59 | override def createDataStreamWriter(df: DataFrame): DataStreamWriter[Row] = {
60 | df
61 | .writeStream
62 | .outputMode(OutputMode.Append)
63 | .format("csv")
64 | .trigger(Trigger.ProcessingTime(s"$triggerSeconds seconds"))
65 | .option("checkpointLocation", checkpointLocation(name))
66 | .option("path", s"$outputDir/$name.csv")
67 | .option("header", "true")
68 | }
69 |
70 | override val name: String = sinkName
71 | }
72 |
73 | /**
74 | * Purges the output directory.
75 | */
76 | def cleanOutputs(): Unit = {
77 | val file = new File(outputDir)
78 | if (file.exists())
79 | file.listFiles().foreach {
80 | FileUtils.deleteDirectory
81 | }
82 | logger.info(s"Cleaned output directory $outputDir")
83 | }
84 | }
85 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/scala/com/cloudera/streaming/refapp/IntegrationTestBase.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import java.util.UUID
7 |
8 | import org.scalatest.AppendedClues._
9 | import org.scalatest.Matchers._
10 | import org.scalatest.concurrent.Eventually.{eventually, interval, timeout}
11 | import org.scalatest.time.{Millis, Seconds, Span}
12 | import org.scalatest.{BeforeAndAfter, FunSuite}
13 | import org.apache.spark.sql.streaming.StreamingQuery
14 | import org.slf4j.LoggerFactory
15 |
16 |
17 | abstract class IntegrationTestBase extends FunSuite with BeforeAndAfter {
18 |
19 | private val logger = LoggerFactory.getLogger(getClass)
20 |
21 | private var diagnosticQuery: Option[StreamingQuery] = None
22 |
23 | before {
24 | waitForOneMessage()
25 | }
26 |
27 | after {
28 | diagnosticQuery.foreach(query => query.stop())
29 | }
30 |
31 | private def waitForOneMessage() = {
32 | EmbeddedKafkaBroker.start()
33 |
34 | val spark = EmbeddedSpark.sparkSession
35 |
36 | val topicName = UUID.randomUUID().toString.replaceAll("-", "_")
37 |
38 | val source = new KafkaSource(spark, EmbeddedKafkaBroker.defaultKafkaConfig)
39 | .stringStream(topicName)
40 |
41 | val query = Memory.memorySink(topicName).createDataStreamWriter(source).start()
42 | diagnosticQuery = Some(query)
43 |
44 | EmbeddedKafkaBroker.publishStringMessageToKafka(topicName, "test")
45 |
46 | eventually(timeout(Span(5, Seconds)), interval(Span(5, Millis))) {
47 | query.processAllAvailable()
48 | val currentContent = spark.table(topicName).collect().map(row => row.getAs[String]("value"))
49 |
50 | currentContent.shouldBe(Array("test")).
51 | withClue("Spark did not get diagnostic message from Kafka. Either one of them failed to start or they can't communicate.")
52 | }
53 | logger.info("Kafka and Spark are running, they are able to communicate")
54 | }
55 | }
56 |
57 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/scala/com/cloudera/streaming/refapp/LocalIntegrationTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import java.sql.Timestamp
7 |
8 | import org.scalatest.Matchers._
9 | import org.scalatest.concurrent.Eventually._
10 | import org.scalatest.time.{Seconds, Span}
11 |
12 | import org.apache.spark.sql.Encoders
13 |
14 | class LocalIntegrationTest extends IntegrationTestBase {
15 |
16 | test("Integration test with one kafka and one spark instance embedded in the same JVM") {
17 |
18 | val inputDir = "src/test/resources/samples"
19 |
20 | val spark = EmbeddedSpark.sparkSession
21 |
22 | val fileSource = new FileSources(spark, inputDir)
23 | val kafkaConfig = EmbeddedKafkaBroker.defaultKafkaConfig
24 | val kafkaSource = new KafkaSource(spark, kafkaConfig)
25 |
26 | val application = new Application(
27 | spark,
28 | Sources(
29 | statesFromCluster = fileSource.jsonFile("states"),
30 | customersFromCluster = fileSource.jsonFile("customers"),
31 | vendorsFromCluster = fileSource.jsonFile("vendors"),
32 | customersFromStream = kafkaSource.jsonStreamWithKafkaTimestamp("customer"),
33 | vendorsFromStream = kafkaSource.jsonStreamWithTimestampFromMessage("vendor", "update_timestamp"),
34 | transactionsFromStream = kafkaSource.jsonStreamWithTimestampFromMessage("transaction", "event_timestamp")
35 | ),
36 | Sinks(
37 | invalidTransactions = Memory.memorySink("invalidTransactions"),
38 | validTransactions = Memory.memorySink("validTransactions"),
39 | customerOrphans = Memory.memorySink("customerOrphans"),
40 | vendorOrphans = Memory.memorySink("vendorOrphans"),
41 | customers = Memory.memorySink("customers"),
42 | vendors = Memory.memorySink("vendors"),
43 | transactionsOperationalMetadata = Memory.memorySink("transactionsOperationalMetadata")
44 | ))
45 |
46 | application.start()
47 |
48 | eventually(timeout(Span(20, Seconds)), interval(Span(5, Seconds))) {
49 | EmbeddedKafkaBroker.publishStringMessageToKafka(
50 | "transaction",
51 | """{
52 | "transaction_id": "1",
53 | "customer_id": 1,
54 | "vendor_id": 1,
55 | "event_state": "CREATED",
56 | "event_timestamp": "2018-11-12 09:42:00",
57 | "price": "100",
58 | "card_type": "Credit"}""")
59 | EmbeddedKafkaBroker.publishStringMessageToKafka(
60 | "transaction",
61 | """{
62 | "transaction_id": "21",
63 | "customer_id": 100,
64 | "vendor_id": 2,
65 | "event_state": "SWIPED",
66 | "event_timestamp": "2018-11-13 09:45:01",
67 | "price": "100",
68 | "card_type": "Debit"}""")
69 |
70 | val validTransactionsQuery = application.streamingQueries.validTransactions
71 | validTransactionsQuery.processAllAvailable()
72 | val currentContent = spark.table("validTransactions").as[Transaction](Encoders.product).collect()
73 |
74 | currentContent.shouldBe(
75 | Array(
76 | Transaction(
77 | transaction_id = "1",
78 | customer_id = Some(1),
79 | vendor_id = Some(1),
80 | event_state = Some("CREATED"),
81 | event_timestamp = Timestamp.valueOf("2018-11-12 09:42:00"),
82 | price = Some("100"),
83 | card_type = Some("Credit")),
84 | Transaction(
85 | transaction_id = "21",
86 | customer_id = Some(100),
87 | vendor_id = Some(2),
88 | event_state = Some("SWIPED"),
89 | event_timestamp = Timestamp.valueOf("2018-11-13 09:45:01"),
90 | price = Some("100"),
91 | card_type = Some("Debit"))
92 | ))
93 | }
94 | }
95 | }
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/scala/com/cloudera/streaming/refapp/Memory.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import org.apache.spark.sql.streaming.{DataStreamWriter, OutputMode}
7 | import org.apache.spark.sql.{DataFrame, Row}
8 |
9 | object Memory {
10 |
11 | def memorySink(sinkName: String) = new Sink {
12 | override def createDataStreamWriter(df: DataFrame): DataStreamWriter[Row] = df
13 | .writeStream
14 | .outputMode(OutputMode.Append)
15 | .queryName(name)
16 | .format("memory")
17 |
18 | override val name: String = sinkName
19 | }
20 |
21 | }
22 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/scala/com/cloudera/streaming/refapp/StructuredStreams.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import java.time.Duration
7 |
8 | object StructuredStreams {
9 |
10 | def main(args: Array[String]) {
11 |
12 | if (args.length != 3) {
13 | sys.error(
14 | """Usage:
15 | |com.cloudera.streaming.refapp.StructuredStreams inputDir outputDir kudu-master
16 | |inputDir should have the same structure as the src/main/resources/samples directory of this project
17 | |outputDir is created if it does not exist and it's purged if it exists
18 | |kudu-master host:port pair pointing to a kudu master instance""".stripMargin)
19 | }
20 | val Array(inputDir, outputDir, kuduMaster) = args
21 |
22 | val spark = EmbeddedSpark.sparkSession
23 |
24 | val fileSource = new FileSources(spark, inputDir)
25 | val fileSink = new FileSinks(outputDir, defaultCheckpointLocation)
26 | val kafkaConfig = EmbeddedKafkaBroker.defaultKafkaConfig
27 | val kafkaSource = new KafkaSource(spark, kafkaConfig)
28 |
29 | val kuduDatabase = "streaming_ref"
30 | val kuduSource = new KuduSource(spark, kuduMaster, kuduDatabase)
31 | val kuduSink = new KuduSink(kuduMaster, kuduDatabase, defaultCheckpointLocation)
32 |
33 | val application = new Application(
34 | spark,
35 | Sources(
36 | statesFromCluster = kuduSource.loadTable("states"),
37 | customersFromCluster = fileSource.jsonFile("customers"),
38 | vendorsFromCluster = kuduSource.loadTable("vendors"),
39 | // customersFromStream = fileSource.jsonStream("customers", "update_timestamp"),
40 | // vendorsFromStream = fileSource.jsonStream("vendors", "update_timestamp"),
41 | // transactionsFromStream = fileSource.jsonStream("transactions", "event_timestamp")
42 | customersFromStream = kafkaSource.jsonStreamWithKafkaTimestamp("customer"),
43 | vendorsFromStream = kafkaSource.jsonStreamWithTimestampFromMessage("vendor", "update_timestamp"),
44 | transactionsFromStream = kafkaSource.jsonStreamWithTimestampFromMessage("transaction", "event_timestamp")
45 | ),
46 | Sinks(
47 | // invalidTransactions = fileSink.csv("invalidTransactions"),
48 | // validTransactions = fileSink.csv("validTransactions"),
49 | // customerOrphans = fileSink.csv("customerOrphans"),
50 | // vendorOrphans = fileSink.csv("vendorOrphans"),
51 | // customers = fileSink.csv("customers"),
52 | // vendors = fileSink.csv("vendors"),
53 | // transactionsOperationalMetadata = fileSink.csv("transactionsOperationalMetadata")
54 | validTransactions = kuduSink.writeTable("valid_transactions"),
55 | invalidTransactions = kuduSink.writeTable("invalid_transactions"),
56 | customerOrphans = kuduSink.writeTable("customer_orphans"),
57 | vendorOrphans = kuduSink.writeTable("vendor_orphans"),
58 | customers = kuduSink.writeTable("customers"),
59 | vendors = kuduSink.writeTable("vendors"),
60 | transactionsOperationalMetadata = kuduSink.writeTable("transactions_operational_metadata")
61 | ),
62 | clusterStartup = EmbeddedKafkaBroker.start(),
63 | initSources = {
64 | CustomerGenerator(kafkaConfig, "customer").start()
65 | VendorGenerator(kafkaConfig, "vendor").start()
66 | TransactionGenerator(kafkaConfig, "transaction").start()
67 | },
68 | cleanOutput = fileSink.cleanOutputs,
69 | queryRestartDurations = Map("valid_transactions" -> Duration.ofMinutes(1))
70 | )
71 |
72 | application.start()
73 | spark.streams.awaitAnyTermination()
74 | }
75 | }
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/scala/com/cloudera/streaming/refapp/TransactionsFlowUnitTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import java.sql.Timestamp
7 |
8 | import org.scalatest.BeforeAndAfter
9 |
10 | import org.apache.spark.sql.execution.streaming.MemoryStream
11 |
12 | class TransactionsFlowUnitTest extends UnitTestBase with BeforeAndAfter {
13 | import testImplicits._
14 |
15 | var transactionsFromStream: MemoryStream[Transaction] = _
16 | var transactiosnFlow: TransactionsFlow = _
17 |
18 | before {
19 | transactionsFromStream = MemoryStream[Transaction]
20 | transactiosnFlow = new TransactionsFlow(
21 | spark,
22 | statesFromCluster,
23 | customersFromCluster,
24 | vendorsFromCluster,
25 | transactionsFromStream = transactionsFromStream
26 | .toDF.withColumn("timestamp", $"event_timestamp".cast("timestamp")))
27 | }
28 |
29 | test("Valid records are written to the validTransactions output") {
30 |
31 | val validTransaction = Transaction(
32 | transaction_id = "1",
33 | customer_id = Some(1),
34 | vendor_id = Some(1),
35 | event_state = Some("CREATED"),
36 | event_timestamp = Timestamp.valueOf("2018-11-12 09:42:00"),
37 | price = Some("100"),
38 | card_type = Some("Credit"))
39 |
40 | testStream(transactiosnFlow.validTransactions.select('transaction_id, 'customer_id, 'vendor_id, 'event_state, 'event_timestamp, 'price, 'card_type)) (
41 | AddData(transactionsFromStream, validTransaction),
42 | CheckAnswer(validTransaction)
43 | )
44 | }
45 |
46 | test("Invalid records are written to the invalidTransactions output") {
47 | // Note: transactionsFlow.validTransactions and invalidTransactions contain the fields that we used for internal calculations, e.g. for validation
48 | // It enables us to check the internal calculations
49 | testStream(transactiosnFlow.invalidTransactions.select('transaction_id, 'valid_card_type)) (
50 | AddData(transactionsFromStream,
51 | Transaction(
52 | transaction_id = "2",
53 | customer_id = Some(1),
54 | vendor_id = Some(1),
55 | event_state = Some("CREATED"),
56 | event_timestamp = Timestamp.valueOf("2018-11-12 09:42:00"),
57 | price = Some("100"),
58 | card_type = Some("Invalid"))),
59 | CheckAnswer(("2", false))
60 | )
61 | }
62 |
63 | }
64 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/scala/com/cloudera/streaming/refapp/UnitTestBase.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import java.sql.Timestamp
7 |
8 | import org.scalatest.BeforeAndAfter
9 |
10 | import org.apache.spark.sql.streaming.StreamTest
11 | import org.apache.spark.sql.test.SharedSQLContext
12 |
13 | abstract class UnitTestBase extends StreamTest with BeforeAndAfter with SharedSQLContext {
14 | import testImplicits._
15 |
16 | lazy val statesFromCluster = List(
17 | State(state_id = 1, state_name = "Alabama", state_abbreviation = "AL"),
18 | State(state_id = 2, state_name = "Alaska", state_abbreviation = "AK")).toDF
19 |
20 | lazy val customersFromCluster = List(
21 | Customer(customer_id = 1, first_name = "John", last_name = "Doe", state_id = 1, update_timestamp = Timestamp.valueOf("2018-01-01 01:02:03")),
22 | Customer(customer_id = 2, first_name = "Jane", last_name = "Miller", state_id = 2, update_timestamp = Timestamp.valueOf("2018-01-02 01:02:03"))).toDF
23 |
24 | lazy val vendorsFromCluster = List(
25 | Vendor(vendor_id = 1, vendor_name = "Apple", phone_number = "123456", update_timestamp = Timestamp.valueOf("2018-11-13 01:02:03")),
26 | Vendor(vendor_id = 2, vendor_name = "Dell", phone_number = "345678", update_timestamp = Timestamp.valueOf("2018-11-13 01:02:03"))
27 | ).toDF
28 |
29 | after {
30 | sqlContext.streams.active.foreach(_.stop())
31 | }
32 |
33 | }
34 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/advancedApp/src/test/scala/com/cloudera/streaming/refapp/kudu/KuduSinkUnitTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp.kudu
5 |
6 | import org.apache.kudu.spark.kudu.KuduContext
7 | import org.apache.spark.sql.DataFrame
8 | import org.mockito.Mockito._
9 | import org.scalatest._
10 | import org.scalatest.mockito.MockitoSugar
11 |
12 | class KuduSinkUnitTest extends FunSuite with MockitoSugar {
13 |
14 | private val frame = mock[DataFrame]
15 |
16 | private def setupKuduContextMock(kuduContext: KuduContext, failTimes: Int): KuduContext = {
17 | if (failTimes > 0) {
18 | val stubber = doThrow(new RuntimeException)
19 | for (_ <- 2 to failTimes) {
20 | stubber.doThrow(new RuntimeException)
21 | }
22 | stubber.doCallRealMethod()
23 | .when(kuduContext).upsertRows(frame, "table")
24 | }
25 | kuduContext
26 | }
27 |
28 | test("kudu upsert fails, retries once") {
29 | val helper = new KuduSinkWithMockedContext(setupKuduContextMock(mock[KuduContext], failTimes = 1), 1)
30 |
31 | helper.sink.addBatch(0, frame)
32 | assert(helper.initialized == 1, "context should be initialized once")
33 | }
34 |
35 | test("kudu upsert fails twice, retries once, fails") {
36 | val helper = new KuduSinkWithMockedContext(setupKuduContextMock(mock[KuduContext], failTimes = 2), 1)
37 |
38 | intercept[RuntimeException] {
39 | helper.sink.addBatch(0, frame)
40 | }
41 | assert(helper.initialized == 1, "context should be initialized once")
42 | }
43 |
44 | test("kudu upsert fails 3 times, retries 3 times") {
45 | val helper = new KuduSinkWithMockedContext(setupKuduContextMock(mock[KuduContext], failTimes = 3), 3)
46 | helper.sink.addBatch(0, frame)
47 | assert(helper.initialized == 3, "context should be initialized three times")
48 | }
49 |
50 | test("kudu upsert fails 3 times, retries 4 times") {
51 | val helper = new KuduSinkWithMockedContext(setupKuduContextMock(mock[KuduContext], failTimes = 3), 4)
52 | helper.sink.addBatch(0, frame)
53 | assert(helper.initialized == 3, "context should be initialized only three times")
54 | }
55 |
56 | }
57 |
58 | class KuduSinkWithMockedContext(kuduContext: KuduContext, retries: Int) {
59 |
60 | // KuduSink constructor inits once
61 | var initialized = -1
62 |
63 | private def initKuduConext: KuduContext = {
64 | initialized += 1
65 | kuduContext
66 | }
67 |
68 | val sink = new KuduSink(initKuduConext, Map(
69 | "kudu.table" -> "table",
70 | "kudu.master" -> "master",
71 | "retries" -> retries.toString))
72 | }
73 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/docs/images/dag.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/cloudera/kafka-examples/4408ba67c21a0646ed76b8fd066eca0ef0a311a2/StructuredStreamingRefApp/docs/images/dag.png
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/docs/images/flows.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/cloudera/kafka-examples/4408ba67c21a0646ed76b8fd066eca0ef0a311a2/StructuredStreamingRefApp/docs/images/flows.png
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/docs/images/pipeline.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/cloudera/kafka-examples/4408ba67c21a0646ed76b8fd066eca0ef0a311a2/StructuredStreamingRefApp/docs/images/pipeline.png
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/docs/images/streaming-systems.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/cloudera/kafka-examples/4408ba67c21a0646ed76b8fd066eca0ef0a311a2/StructuredStreamingRefApp/docs/images/streaming-systems.png
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/scripts/config.sh:
--------------------------------------------------------------------------------
1 | # Copyright (C) Cloudera, Inc. 2019
2 |
3 | # Configuration for setup scripts
4 | # Should be sourced by them
5 |
6 | : "Kerberos principal used for setup: ${SETUP_PRINCIPAL:=kafka}"
7 |
8 | : "Keytab used to login as setup principal: ${SETUP_KEYTAB:=/cdep/keytabs/$SETUP_PRINCIPAL.keytab}"
9 |
10 | : "Input topics to set up: ${TOPICS:=customer vendor transaction}"
11 |
12 | : "Zookeeper quorum: ${ZOOKEEPER_QUORUM:=$(hostname):2181}"
13 |
14 | : "Kafka broker list: ${BROKER_LIST:=$(hostname):9093}"
15 |
16 | # Producer
17 | : "Hosts producer connects from: ${PRODUCER_HOSTS:=*}"
18 |
19 | : "Producer user/principal: ${PRODUCER_USER:=flume}"
20 |
21 | : "Keytab used to login as producer: ${PRODUCER_KEYTAB:=/cdep/keytabs/$PRODUCER_USER.keytab}"
22 |
23 | : "Kerberos realm: ${PRODUCER_REALM:=`klist -kt $PRODUCER_KEYTAB | grep '@' | head -n 1 | sed s/.*@// | sed s/[[:space:]]//`}"
24 |
25 | : "Primary group of producer user: ${PRODUCER_GROUP:=`id -gn $PRODUCER_USER`}"
26 |
27 | : "Producer Sentry role: ${PRODUCER_ROLE:=$PRODUCER_GROUP}"
28 |
29 | : "Producer truststore location: ${PRODUCER_TRUSTSTORE_LOCATION:=/etc/cdep-ssl-conf/CA_STANDARD/truststore.jks}"
30 |
31 | : "Directory to store producer's files: ${PRODUCER_FILES_DIR:=`pwd`}"
32 |
33 |
34 | # Spark application as Kafka consumer
35 | : "Hosts Spark application consumes from: ${SPARK_HOSTS:=*}"
36 |
37 | : "Producer user/principal: ${SPARK_USER:=systest}"
38 |
39 | : "Keytab used by Spark application: ${SPARK_KEYTAB:=/cdep/keytabs/$SPARK_USER.keytab}"
40 |
41 | : "Kerberos realm: ${SPARK_REALM:=`klist -kt $SPARK_KEYTAB | grep '@' | head -n 1 | sed s/.*@// | sed s/[[:space:]]//`}"
42 |
43 | : "Primary group of consumer user: ${SPARK_GROUP:=`id -gn $SPARK_USER`}"
44 |
45 | : "Consumer Sentry role: ${SPARK_ROLE:=$SPARK_GROUP}"
46 |
47 | : "Consumer truststore location: ${SPARK_TRUSTSTORE_LOCATION:=/etc/cdep-ssl-conf/CA_STANDARD/truststore.jks}"
48 |
49 | : "Directory to store consumer's files: ${CONSUMER_FILES_DIR:=`pwd`}"
50 |
51 | # Database
52 | : "Database admin user: ${DB_ADMIN_USER:=impala}"
53 |
54 | : "Keytab used to login as db admin user: ${DB_ADMIN_KEYTAB:=/cdep/keytabs/$DB_ADMIN_USER.keytab}"
55 |
56 | : "Impala daemon to connect: ${IMPALA_DAEMON}"
57 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/scripts/kudu.sh:
--------------------------------------------------------------------------------
1 | #!/bin/bash
2 | # Copyright (C) Cloudera, Inc. 2019
3 |
4 | set -fex
5 |
6 | # Creates the streaming_ref datbase and the tables in Kudu and sentry privileges required to access them
7 |
8 | . $(dirname $0)/config.sh
9 |
10 | SQL_FILE=$(dirname $0)/init_kudu_db.sql
11 |
12 | if [ -z ${IMPALA_DAEMON} ]
13 | then
14 | : "\${IMPALA_DAEMON} must be set to the hots[:port] value the shell can connect to"
15 | exit 1
16 | fi
17 |
18 | # Create sentry role for Spark if it does not exist
19 | kinit -kt $SETUP_KEYTAB $SETUP_PRINCIPAL
20 | if kafka-sentry -lr | grep -q $SPARK_ROLE ; then
21 | echo $SPARK_ROLE already exists
22 | else
23 | : "Create sentry role $SPARK_ROLE"
24 | kafka-sentry -cr -r $SPARK_ROLE
25 | fi
26 |
27 | kinit -kt $DB_ADMIN_KEYTAB $DB_ADMIN_USER
28 |
29 | impala-shell -i ${IMPALA_DAEMON} -f ${SQL_FILE} -k --ssl
30 |
31 | impala-shell -i ${IMPALA_DAEMON} -k --ssl -q "GRANT ALL ON DATABASE streaming_ref to ${SPARK_ROLE}"
32 |
33 | kdestroy
34 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/scripts/producer.sh:
--------------------------------------------------------------------------------
1 | #!/bin/bash
2 | # Copyright (C) Cloudera, Inc. 2019
3 |
4 | set -fex
5 |
6 | # Creates sentry privileges and configuration files for producer application
7 |
8 | . $(dirname $0)/config.sh
9 |
10 | kinit -kt $SETUP_KEYTAB $SETUP_PRINCIPAL
11 | : "Create sentry role $PRODUCER_ROLE"
12 | kafka-sentry -cr -r $PRODUCER_ROLE
13 |
14 |
15 | : "Add role $PRODUCER_ROLE to group $PRODUCER_GROUP"
16 | kafka-sentry -arg -r $PRODUCER_ROLE -g $PRODUCER_GROUP
17 |
18 | for TOPIC in $TOPICS
19 | do
20 | for PRODUCER_HOST in $PRODUCER_HOSTS
21 | do
22 | : "Grant privileges to role $PRODUCER_ROLE on topic $TOPIC from host $PRODUCER_HOST"
23 | kafka-sentry -gpr -r $PRODUCER_ROLE -p "Host=$PRODUCER_HOST->Topic=$TOPIC->action=describe"
24 | kafka-sentry -gpr -r $PRODUCER_ROLE -p "Host=$PRODUCER_HOST->Topic=$TOPIC->action=write"
25 | done
26 | done
27 |
28 | : "Create client.properties"
29 | cat >$PRODUCER_FILES_DIR/producer.properties<Topic=$TOPIC->action=describe"
28 | kafka-sentry -gpr -r $SPARK_ROLE -p "Host=$SPARK_HOST->Topic=$TOPIC->action=read"
29 | : "Allow role $SPARK_ROLE to join any consumer group from host $SPARK_HOST"
30 | kafka-sentry -gpr -r $SPARK_ROLE -p "Host=$SPARK_HOST->Consumergroup=*->action=describe"
31 | kafka-sentry -gpr -r $SPARK_ROLE -p "Host=$SPARK_HOST->Consumergroup=*->action=read"
32 | done
33 | done
34 |
35 | : "Create jaas.config"
36 | cat > kafka_client_jaas.conf<$CONSUMER_FILES_DIR/consumer.properties< -k --ssl`
54 | * in the shell execute
55 | ```
56 | use streaming_ref;
57 |
58 | select count(*) from transactions
59 | ```
60 | to quickly check if the application is producing output, or you can execute any other queries against the output tables.
61 |
62 |
63 | #### Submitting the application without security
64 |
65 | Execute
66 |
67 | ```
68 | spark-submit --files consumer.properties \
69 | --class com.cloudera.streaming.refapp.StructuredStreamingApp --deploy-mode cluster \
70 | --master yarn streaming-ref-app-simple-0.1-SNAPSHOT-jar-with-dependencies.jar \
71 | consumer.properties
72 | ```
73 |
74 | #### Submitting the application on a secured cluster
75 |
76 | Execute
77 |
78 | ```
79 | kinit -kt
80 |
81 | spark-submit --files consumer.properties,kafka_client_jaas.conf, --driver-java-options \
82 | "-Djava.security.auth.login.config=./kafka_client_jaas.conf" --class com.cloudera.streaming.refapp.StructuredStreamingApp \
83 | --conf "spark.executor.extraJavaOptions=-Djava.security.auth.login.config=./kafka_client_jaas.conf" \
84 | --deploy-mode cluster --master yarn streaming-ref-app-simple-0.1-SNAPSHOT-jar-with-dependencies.jar \
85 | consumer.properties
86 | ```
87 |
88 | The application will keep running until you kill it in yarn.
89 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/simpleApp/db/init_kudu_db.sql:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 |
5 | CREATE DATABASE IF NOT EXISTS streaming_ref;
6 | USE streaming_ref;
7 |
8 | DROP TABLE IF EXISTS customers;
9 | CREATE TABLE customers (
10 | customer_id INT PRIMARY KEY,
11 | first_name STRING,
12 | last_name STRING,
13 | state_name STRING,
14 | state_abbreviation STRING,
15 | update_timestamp TIMESTAMP)
16 | PARTITION BY HASH (customer_id) PARTITIONS 10
17 | STORED AS KUDU TBLPROPERTIES ('kudu.num_tablet_replicas' = '3');
18 |
19 | DROP TABLE IF EXISTS transactions;
20 | CREATE TABLE transactions (
21 | event_timestamp TIMESTAMP,
22 | transaction_id STRING,
23 | vendor_id INT,
24 | event_state STRING,
25 | price STRING,
26 | card_type STRING,
27 | customer_id INT,
28 | customer_first_name STRING,
29 | customer_last_name STRING,
30 | is_valid BOOLEAN,
31 | PRIMARY KEY (event_timestamp, transaction_id)
32 | )
33 | PARTITION BY
34 | HASH (transaction_id) PARTITIONS 15,
35 | RANGE (event_timestamp)
36 | (PARTITION '2018-11-01' <= VALUES < '2018-12-01')
37 | STORED AS KUDU TBLPROPERTIES ('kudu.num_tablet_replicas' = '1');
38 | ALTER TABLE transactions ADD RANGE PARTITION '2018-12-01' <= VALUES < '2019-01-01';
39 | ALTER TABLE transactions ADD RANGE PARTITION '2019-01-01' <= VALUES < '2019-02-01';
40 | ALTER TABLE transactions ADD RANGE PARTITION '2019-02-01' <= VALUES < '2019-03-01';
41 | -- ...
42 |
43 | DROP TABLE IF EXISTS operational_metadata;
44 | CREATE TABLE operational_metadata(
45 | start_ts TIMESTAMP PRIMARY KEY,
46 | end_ts TIMESTAMP,
47 | num_transactions BIGINT)
48 | STORED AS KUDU TBLPROPERTIES ('kudu.num_tablet_replicas' = '3');
49 |
50 | insert into customers values (1, 'John', 'Doe', 'Alabama', 'AL', '2018-01-01');
51 | insert into customers values (2, 'Jane', 'Miller', 'Alaska', 'AK', '2018-01-01');
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/simpleApp/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
5 | 4.0.0
6 |
7 | com.cloudera.streaming.examples
8 | streaming-ref-app-simple
9 | 0.1-SNAPSHOT
10 |
11 |
12 |
13 | cloudera
14 | https://repository.cloudera.com/artifactory/cloudera-repos/
15 |
16 |
17 |
18 |
19 | 2.11
20 | 1.8
21 | 3.3.2
22 | 3.7.0
23 | 2.4.0-cdh6.1.0
24 | 2.0.0-cdh6.1.0
25 | 1.8.0-cdh6.1.0
26 |
27 |
28 |
29 |
30 | org.apache.spark
31 | spark-sql-kafka-0-10_${scala.version}
32 | ${spark.version}
33 | provided
34 |
35 |
36 | org.apache.spark
37 | spark-sql_${scala.version}
38 | ${spark.version}
39 | provided
40 |
41 |
42 | org.apache.kudu
43 | kudu-spark2_${scala.version}
44 | ${kudu.version}
45 |
46 |
47 |
48 |
49 |
50 |
51 | net.alchim31.maven
52 | scala-maven-plugin
53 | ${scala.maven.plugin.version}
54 |
55 |
56 |
57 | compile
58 | testCompile
59 |
60 |
61 |
62 |
63 |
64 | maven-compiler-plugin
65 | ${maven.compiler.plugin.version}
66 |
67 | ${java.version}
68 | ${java.version}
69 |
70 |
71 |
72 | maven-assembly-plugin
73 |
74 |
75 | jar-with-dependencies
76 |
77 |
78 |
79 |
80 | make-assembly
81 | package
82 |
83 | single
84 |
85 |
86 |
87 |
88 |
89 |
90 |
91 |
92 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/simpleApp/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister:
--------------------------------------------------------------------------------
1 | com.cloudera.streaming.refapp.kudu.KuduSinkProvider
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/simpleApp/src/main/scala/com/cloudera/streaming/refapp/StructuredStreamingApp.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright (C) Cloudera, Inc. 2019
3 | */
4 | package com.cloudera.streaming.refapp
5 |
6 | import java.io.{FileInputStream, InputStream}
7 | import java.sql.Timestamp
8 | import java.util.{Properties, UUID}
9 |
10 | import scala.collection.JavaConverters._
11 |
12 | import org.apache.kudu.spark.kudu._
13 |
14 | import org.apache.spark.sql.types.StructType
15 | import org.apache.spark.sql.{DataFrame, Encoders, SparkSession, functions}
16 |
17 | /**
18 | * A long running streaming application that can be submitted to a running Spark service.
19 | * It reads static data from Kudu, streaming data from Kafka and writes output to Kudu.
20 | * It demonstrates
21 | * - joining the two datasets
22 | * - performing simple validation
23 | * - working with time windows to create aggregate statistics
24 | * It focuses on the kafka-spark-kudu integration and keeps all other aspects as simple as possible.
25 | */
26 | object StructuredStreamingApp {
27 |
28 | case class Transaction(transaction_id: String,
29 | customer_id: Option[Int],
30 | vendor_id: Option[Int],
31 | event_state: Option[String],
32 | event_timestamp: Timestamp,
33 | price: Option[String],
34 | card_type: Option[String])
35 |
36 | def main(args: Array[String]) {
37 |
38 | if (args.length < 2) {
39 | sys.error(
40 | """Usage:
41 | |com.cloudera.streaming.refapp.StructuredStreamingApp consumer.config kudu-master
42 | |consumer.config path to kafka client properties
43 | |kudu-master host:port pair pointing to a kudu master instance
44 | """.stripMargin)
45 | }
46 |
47 | // extract first two arguments
48 | val Array(consumerConfig, kuduMaster) = args
49 | val kafkaParamsForSpark = kafkaConfigFromPropertiesFile(consumerConfig)
50 |
51 | val spark = SparkSession.builder().appName("streaming-ref").getOrCreate()
52 |
53 | import spark.implicits._
54 |
55 | def readKafkaStream(topic: String, schema: StructType) = {
56 | val kafkaOptions = kafkaParamsForSpark ++ Map("subscribe" -> topic, "startingoffsets" -> "latest")
57 |
58 | val df = spark.readStream.format("kafka").options(kafkaOptions).load()
59 | .selectExpr("CAST(value AS STRING)")
60 | .select(functions.from_json('value, schema) as "parsedValue")
61 | .selectExpr("parsedValue.*")
62 | df.createOrReplaceTempView(topic)
63 | df
64 | }
65 |
66 | val kuduDatabase = "streaming_ref"
67 |
68 | def readKuduTable(name: String) = {
69 | val fullTableName = s"impala::$kuduDatabase.$name"
70 | val df = spark
71 | .read
72 | .options(Map(
73 | "kudu.master" -> kuduMaster,
74 | "kudu.table" -> fullTableName))
75 | .kudu
76 | df.createOrReplaceTempView(name)
77 | df
78 | }
79 |
80 | // Checkpointing is needed for failure handling: streaming queries that use checkpointing
81 | // can be continued after a faileru where the failed one left off, ensuring data consistency guarantees.
82 | // Each query needs a unique checkpoint location, that's why a random UUID is used.
83 | // In production you may want to set it to a stable, but unique, reliable location (e.g. on HDFS).
84 | val baseCheckpointLocation = "/tmp/temporary-" + UUID.randomUUID.toString
85 |
86 | def writeKuduTable(df: DataFrame, name: String) = {
87 | val fullTableName = s"impala::$kuduDatabase.$name"
88 | df
89 | .writeStream
90 | .format("kudu")
91 | .option("kudu.master", kuduMaster)
92 | .option("kudu.table", fullTableName)
93 | .option("checkpointLocation", s"$baseCheckpointLocation/$name")
94 | .option("retries", "3")
95 | .outputMode("update")
96 | .start()
97 | }
98 |
99 | val customers = readKuduTable("customers")
100 |
101 | val transactions = readKafkaStream("transaction", Encoders.product[Transaction].schema)
102 | .withWatermark("event_timestamp", "1 minute")
103 |
104 | val enrichedTransactions = spark.sql(
105 | """
106 | SELECT
107 | t.*,
108 | c.first_name as customer_first_name, c.last_name as customer_last_name,
109 | card_type in ('Visa', 'MasterCard') and event_state in ('created') as is_valid
110 | FROM transaction t
111 | LEFT OUTER JOIN customers c ON t.customer_id = c.customer_id
112 | """.stripMargin)
113 |
114 | writeKuduTable(enrichedTransactions, "transactions")
115 |
116 | val operationalMetadata = transactions
117 | .groupBy(functions.window(functions.col("event_timestamp"), "1 minutes"))
118 | .count().as("c")
119 | .selectExpr("c.window.start as start_ts", "c.window.end as end_ts", "c.count as num_transactions")
120 |
121 | writeKuduTable(operationalMetadata, "operational_metadata")
122 |
123 | spark.streams.awaitAnyTermination()
124 |
125 | }
126 |
127 | def kafkaConfigFromPropertiesFile(configFile: String) = {
128 | var inputStream: Option[InputStream] = None
129 | val kafkaParams = try {
130 | inputStream = Some(new FileInputStream(configFile))
131 | val params = new Properties()
132 | params.load(inputStream.get)
133 | inputStream.get.close()
134 | params.asScala.toMap
135 | } finally {
136 | inputStream.foreach(_.close())
137 | }
138 | kafkaParams.map {
139 | case (key, value) => "kafka." + key -> value
140 | }
141 | }
142 |
143 | }
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/simpleApp/src/main/scala/com/cloudera/streaming/refapp/kudu/KuduSink.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Modifications copyright (C) 2019 Cloudera Inc
3 | */
4 | package com.cloudera.streaming.refapp.kudu
5 |
6 | import org.apache.kudu.spark.kudu.KuduContext
7 | import org.apache.spark.sql.execution.streaming.Sink
8 | import org.apache.spark.sql.{DataFrame, SQLContext}
9 | import org.slf4j.LoggerFactory
10 |
11 | import scala.util.control.NonFatal
12 |
13 | object KuduSink {
14 | def withDefaultContext(sqlContext: SQLContext, parameters: Map[String, String]) =
15 | new KuduSink(new KuduContext(parameters("kudu.master"), sqlContext.sparkContext), parameters)
16 | }
17 |
18 | /**
19 | * A simple Structured Streaming sink which writes the data frame to Kudu.
20 | * It preserves exactly once semantics, as it's idempotent in the face of
21 | * multiple attempts to add the same batch.
22 | *
23 | * It uses the following parameters:
24 | * kudu.master - host:port pair of a kudu master node
25 | * kudu.table - full table name
26 | * checkpointLocation - where the checkpoint will be stored
27 | */
28 | class KuduSink(initKuduContext: => KuduContext, parameters: Map[String, String]) extends Sink {
29 |
30 | private val logger = LoggerFactory.getLogger(getClass)
31 |
32 | private var kuduContext = initKuduContext
33 |
34 | private val tablename = parameters("kudu.table")
35 |
36 | private val retries = parameters.getOrElse("retries", "1").toInt
37 | require(retries >= 0, "retries must be non-negative")
38 |
39 | logger.info(s"Created Kudu sink writing to table $tablename")
40 |
41 | override def addBatch(batchId: Long, data: DataFrame): Unit = {
42 | for (attempt <- 0 to retries) {
43 | try {
44 | kuduContext.upsertRows(data, tablename)
45 | return
46 | } catch {
47 | case NonFatal(e) =>
48 | if (attempt < retries) {
49 | logger.warn("Kudu upsert error, retrying...", e)
50 | kuduContext = initKuduContext
51 | }
52 | else {
53 | logger.error("Kudu upsert error, exhausted", e)
54 | throw e
55 | }
56 | }
57 | }
58 | }
59 | }
60 |
--------------------------------------------------------------------------------
/StructuredStreamingRefApp/simpleApp/src/main/scala/com/cloudera/streaming/refapp/kudu/KuduSinkProvider.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Modifications copyright (C) 2019 Cloudera Inc
3 | */
4 | package com.cloudera.streaming.refapp.kudu
5 |
6 | import org.apache.spark.sql.SQLContext
7 | import org.apache.spark.sql.execution.streaming.Sink
8 | import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider}
9 | import org.apache.spark.sql.streaming.OutputMode
10 |
11 | /**
12 | * Registers KuduSink to Spark streaming, so it can be used with format "kudu".
13 | *
14 | * Note: to make it effective you need META-INF/services/org.apache.spark.sql.sources.DataSourceRegister to
15 | * refer to this class.
16 | */
17 | class KuduSinkProvider extends StreamSinkProvider with DataSourceRegister {
18 |
19 | override def createSink(sqlContext: SQLContext,
20 | parameters: Map[String, String],
21 | partitionColumns: Seq[String],
22 | outputMode: OutputMode): Sink = {
23 | require(outputMode == OutputMode.Update, "only 'update' OutputMode is supported")
24 | KuduSink.withDefaultContext(sqlContext, parameters)
25 | }
26 |
27 | override def shortName(): String = "kudu"
28 | }
--------------------------------------------------------------------------------