├── .gitignore
├── README.txt
├── READMEGUIDE.docx
├── data
└── sensordata.csv
├── pom.xml
└── src
└── main
├── java
└── solution
│ ├── MyConsumer.java
│ └── MyProducer.java
└── scala
├── exercise
└── SensorStreamConsumer.scala
└── solution
├── HBaseReadRowWriteStats.scala
├── HBaseSensorStream.scala
└── SensorStreamConsumer.scala
/.gitignore:
--------------------------------------------------------------------------------
1 | /target/
--------------------------------------------------------------------------------
/README.txt:
--------------------------------------------------------------------------------
1 |
2 | Commands to run :
3 |
4 | Step 1: First compile the project on eclipse: Select project -> Run As -> Maven Install
5 |
6 | Step 2: use scp to copy the ms-sparkstreaming-1.0.jar to the mapr sandbox or cluster
7 |
8 | also use scp to copy the data sensor.csv file from the data folder to the cluster
9 | put this file in a folder called data. The producer reads from this file to send messages.
10 |
11 | scp ms-sparkstreaming-1.0.jar user01@ipaddress:/user/user01/.
12 | if you are using virtualbox:
13 | scp -P 2222 ms-sparkstreaming-1.0.jar user01@127.0.0.1:/user/user01/.
14 |
15 | Create the topic
16 |
17 | maprcli stream create -path /user/user01/pump -produceperm p -consumeperm p -topicperm p
18 | maprcli stream topic create -path /user/user01/pump -topic sensor -partitions 3
19 |
20 | get info on the topic
21 | maprcli stream info -path /user/user01/pump
22 |
23 |
24 | To run the MapR Streams Java producer and consumer:
25 |
26 | java -cp ms-sparkstreaming-1.0.jar:`mapr classpath` solution.MyProducer
27 |
28 | java -cp ms-sparkstreaming-1.0.jar:`mapr classpath` solution.MyConsumer
29 |
30 | Step 3: To run the Spark streaming consumer:
31 |
32 | start the streaming app
33 |
34 | spark-submit --class solution.SensorStreamConsumer --master local[2] ms-sparkstreaming-1.0.jar
35 |
36 | ctl-c to stop
37 |
38 | step 4: Spark streaming app writing to hbase
39 |
40 | first make sure that you have the correct version of HBase installed, it should be 1.1.1:
41 |
42 | cat /opt/mapr/hbase/hbaseversion
43 | 1.1.1
44 |
45 | Next make sure the Spark HBase compatibility version is correctly configured here:
46 | cat /opt/mapr/spark/spark-1.5.2/mapr-util/compatibility.version
47 | hbase_versions=1.1.1
48 |
49 | If this is not 1.1.1 fix it.
50 |
51 |
52 | Create an hbase table to write to:
53 | launch the hbase shell
54 | $hbase shell
55 |
56 | create '/user/user01/sensor', {NAME=>'data'}, {NAME=>'alert'}, {NAME=>'stats'}
57 |
58 | Step 4: start the streaming app writing to HBase
59 |
60 | spark-submit --class solution.HBaseSensorStream --master local[2] ms-sparkstreaming-1.0.jar
61 |
62 | ctl-c to stop
63 |
64 | Scan HBase to see results:
65 |
66 | $hbase shell
67 |
68 | scan '/user/user01/sensor' , {'LIMIT' => 5}
69 |
70 | scan '/user/user01/sensor' , {'COLUMNS'=>'alert', 'LIMIT' => 50}
71 |
72 | Step 5:
73 | exit, run spark (not streaming) app to read from hbase and write daily stats
74 |
75 | spark-submit --class solution.HBaseReadRowWriteStats --master local[2] ms-sparkstreaming-1.0.jar
76 |
77 | $hbase shell
78 |
79 | scan '/user/user01/sensor' , {'COLUMNS'=>'stats', 'LIMIT' => 50}
80 |
81 |
82 | cleanup if you want to re-run:
83 |
84 | maprcli stream topic delete -path /user/user01/pump -topic sensor
85 | maprcli stream topic create -path /user/user01/pump -topic sensor -partitions 3
86 | hbase shell
87 | truncate '/user/user01/sensor'
--------------------------------------------------------------------------------
/READMEGUIDE.docx:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/caroljmcdonald/mapr-streams-sparkstreaming-hbase/e1c912d59c0d3f456bac56c4c75d1e35f07f37dd/READMEGUIDE.docx
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 | 4.0.0
4 | mapr
5 | ms-sparkstreaming
6 | 1.0
7 | jar
8 |
9 | 1.7
10 | 1.7
11 | UTF-8
12 | 2.10
13 | 2.10.4
14 | 1.5.2
15 | 1.1.1-mapr-1602-m7-5.1.0
16 | 2.15.2
17 |
18 |
19 |
20 |
21 | scala-tools.org
22 | Scala-tools Maven2 Repository
23 | http://scala-tools.org/repo-releases
24 |
25 |
26 | mapr-releases
27 | http://repository.mapr.com/maven/
28 |
29 | false
30 |
31 |
32 | true
33 |
34 |
35 |
36 |
37 |
38 | scala-tools.org
39 | Scala-tools Maven2 Repository
40 | http://scala-tools.org/repo-releases
41 |
42 |
43 |
44 |
45 |
46 | org.scala-lang
47 | scala-library
48 | ${scala.version}
49 |
50 |
51 | org.apache.spark
52 | spark-core_${scala.tools.version}
53 | ${spark.version}
54 | provided
55 |
56 |
57 | org.apache.spark
58 | spark-sql_${scala.tools.version}
59 | ${spark.version}
60 |
61 |
62 | org.apache.spark
63 | spark-streaming_${scala.tools.version}
64 | ${spark.version}
65 |
66 |
67 | org.apache.hbase
68 | hbase-server
69 | ${mapr.version}
70 |
71 |
72 | org.apache.spark
73 | spark-streaming-kafka-v09_2.10
74 | 1.5.2-mapr-1602
75 |
76 |
77 | org.apache.spark
78 | spark-streaming-kafka-producer_2.10
79 | 1.5.2-mapr-1602
80 |
81 |
82 | org.apache.kafka
83 | kafka-clients
84 | 0.9.0.0-mapr-1602-streams-5.1.0
85 |
86 |
87 | org.apache.zookeeper
88 | zookeeper
89 |
90 |
91 |
92 |
93 |
94 |
95 |
96 |
97 |
98 | org.scala-tools
99 | maven-scala-plugin
100 | 2.15.2
101 |
102 |
103 |
104 | compile
105 |
106 |
107 |
108 |
109 |
110 |
111 | org.apache.maven.plugins
112 | maven-eclipse-plugin
113 | 2.8
114 |
115 |
116 |
117 | ms-sparkstreaming
118 |
--------------------------------------------------------------------------------
/src/main/java/solution/MyConsumer.java:
--------------------------------------------------------------------------------
1 | /* Copyright (c) 2009 & onwards. MapR Tech, Inc., All rights reserved */
2 | package solution;
3 |
4 | import org.apache.kafka.clients.consumer.ConsumerRecord;
5 | import org.apache.kafka.clients.consumer.ConsumerRecords;
6 | import org.apache.kafka.clients.consumer.KafkaConsumer;
7 |
8 | import java.io.IOException;
9 | import java.util.ArrayList;
10 | import java.util.Iterator;
11 | import java.util.List;
12 | import java.util.Properties;
13 |
14 | public class MyConsumer {
15 |
16 | // Declare a new consumer.
17 | public static KafkaConsumer consumer;
18 |
19 | public static void main(String[] args) throws IOException {
20 | configureConsumer(args);
21 |
22 | String topic = "/user/user01/pump:sensor";
23 | if (args.length == 1) {
24 | topic = args[0];
25 | }
26 |
27 | List topics = new ArrayList();
28 | topics.add(topic);
29 | // Subscribe to the topic.
30 | consumer.subscribe(topics);
31 |
32 | // Set the timeout interval for requests for unread messages.
33 | long pollTimeOut = 1000;
34 | long waitTime = 30 * 1000; // loop for while loop 30 seconds
35 | long numberOfMsgsReceived = 0;
36 | while (waitTime > 0) {
37 | // Request unread messages from the topic.
38 | ConsumerRecords msg = consumer.poll(pollTimeOut);
39 | if (msg.count() == 0) {
40 | System.out.println("No messages after 1 second wait.");
41 | } else {
42 | System.out.println("Read " + msg.count() + " messages");
43 | numberOfMsgsReceived += msg.count();
44 |
45 | // Iterate through returned records, extract the value
46 | // of each message, and print the value to standard output.
47 | Iterator> iter = msg.iterator();
48 | while (iter.hasNext()) {
49 | ConsumerRecord record = iter.next();
50 | System.out.println("Consuming " + record.toString());
51 |
52 | }
53 | }
54 | waitTime = waitTime - 1000; // decrease time for loop
55 | }
56 | consumer.close();
57 | System.out.println("Total number of messages received: " + numberOfMsgsReceived);
58 | System.out.println("All done.");
59 |
60 | }
61 |
62 | /* Set the value for configuration parameters.*/
63 | public static void configureConsumer(String[] args) {
64 | Properties props = new Properties();
65 | // cause consumers to start at beginning of topic on first read
66 | props.put("auto.offset.reset", "earliest");
67 | props.put("key.deserializer",
68 | "org.apache.kafka.common.serialization.StringDeserializer");
69 | // which class to use to deserialize the value of each message
70 | props.put("value.deserializer",
71 | "org.apache.kafka.common.serialization.StringDeserializer");
72 |
73 | consumer = new KafkaConsumer(props);
74 | }
75 |
76 | }
77 |
--------------------------------------------------------------------------------
/src/main/java/solution/MyProducer.java:
--------------------------------------------------------------------------------
1 | /* Copyright (c) 2009 & onwards. MapR Tech, Inc., All rights reserved */
2 | package solution;
3 |
4 | import java.io.BufferedReader;
5 | import java.io.File;
6 | import java.io.FileReader;
7 | import org.apache.kafka.clients.producer.KafkaProducer;
8 | import org.apache.kafka.clients.producer.ProducerRecord;
9 |
10 | import java.io.IOException;
11 | import java.util.Properties;
12 |
13 | public class MyProducer {
14 |
15 | // Set the number of messages to send.
16 | public static int numMessages = 60;
17 | // Declare a new producer
18 | public static KafkaProducer producer;
19 |
20 | public static void main(String[] args) throws IOException {
21 | // Set the stream and topic to publish to.
22 | String topic = "/user/user01/pump:sensor";
23 | if (args.length == 1) {
24 | topic = args[0];
25 | }
26 |
27 | configureProducer();
28 | File f = new File("./data/sensordata.csv");
29 | FileReader fr = new FileReader(f);
30 | BufferedReader reader = new BufferedReader(fr);
31 | String line = reader.readLine();
32 | while (line != null) {
33 | String[] temp = line.split(",");
34 | String key=temp[0];
35 | /* Add each message to a record. A ProducerRecord object
36 | identifies the topic or specific partition to publish
37 | a message to. */
38 | ProducerRecord rec = new ProducerRecord(topic,key, line);
39 |
40 | // Send the record to the producer client library.
41 | System.out.println("Sending to topic " + topic);
42 | producer.send(rec);
43 | System.out.println("Sent message " + line);
44 | line = reader.readLine();
45 |
46 | }
47 |
48 | producer.close();
49 | System.out.println("All done.");
50 |
51 | System.exit(1);
52 |
53 | }
54 |
55 | /* Set the value for a configuration parameter.
56 | This configuration parameter specifies which class
57 | to use to serialize the value of each message.*/
58 | public static void configureProducer() {
59 | Properties props = new Properties();
60 | props.put("key.serializer",
61 | "org.apache.kafka.common.serialization.StringSerializer");
62 | props.put("value.serializer",
63 | "org.apache.kafka.common.serialization.StringSerializer");
64 |
65 | producer = new KafkaProducer(props);
66 | }
67 |
68 | }
69 |
--------------------------------------------------------------------------------
/src/main/scala/exercise/SensorStreamConsumer.scala:
--------------------------------------------------------------------------------
1 | package exercise
2 |
3 | import org.apache.kafka.clients.consumer.ConsumerConfig
4 | import org.apache.kafka.common.serialization.StringDeserializer
5 | import org.apache.spark.{ SparkConf, SparkContext }
6 | import org.apache.spark.SparkContext._
7 | import org.apache.spark.streaming._
8 | import org.apache.spark.streaming.dstream.{ DStream, InputDStream }
9 | import org.apache.spark.streaming.kafka.v09.KafkaUtils
10 | import org.apache.spark.streaming.{ Seconds, StreamingContext }
11 | import org.apache.spark.sql.functions.avg
12 | import org.apache.spark.sql.SQLContext
13 |
14 | object SensorStreamConsumer extends Serializable {
15 |
16 | // schema for sensor data
17 | case class Sensor(resid: String, date: String, time: String, hz: Double, disp: Double, flo: Double, sedPPM: Double, psi: Double, chlPPM: Double) extends Serializable
18 |
19 | // function to parse line of sensor data into Sensor class
20 | def parseSensor(str: String): Sensor = {
21 | val p = str.split(",")
22 | Sensor(p(0), p(1), p(2), p(3).toDouble, p(4).toDouble, p(5).toDouble, p(6).toDouble, p(7).toDouble, p(8).toDouble)
23 | }
24 | val timeout = 10 // Terminate after N seconds
25 | val batchSeconds = 2 // Size of batch intervals
26 |
27 | def main(args: Array[String]): Unit = {
28 |
29 | val brokers = "maprdemo:9092" // not needed for MapR Streams, needed for Kafka
30 | val groupId = "testgroup"
31 | val offsetReset = "earliest"
32 | val batchInterval = "2"
33 | val pollTimeout = "1000"
34 | val topics = "/user/user01/pump:sensor"
35 |
36 | val sparkConf = new SparkConf().setAppName("SensorStream")
37 |
38 | val ssc = new StreamingContext(sparkConf, Seconds(batchInterval.toInt))
39 |
40 | // Create direct kafka stream with brokers and topics
41 | val topicsSet = topics.split(",").toSet
42 | val kafkaParams = Map[String, String](
43 | ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> brokers,
44 | ConsumerConfig.GROUP_ID_CONFIG -> groupId,
45 | ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG ->
46 | "org.apache.kafka.common.serialization.StringDeserializer",
47 | ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG ->
48 | "org.apache.kafka.common.serialization.StringDeserializer",
49 | ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> offsetReset,
50 | ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG -> "false",
51 | "spark.kafka.poll.time" -> pollTimeout
52 | )
53 |
54 | val messages = KafkaUtils.createDirectStream[String, String](ssc, kafkaParams, topicsSet)
55 |
56 | val sensorDStream = messages.map(_._2).map(parseSensor)
57 |
58 | sensorDStream.foreachRDD { rdd =>
59 |
60 | // There exists at least one element in RDD
61 | if (!rdd.isEmpty) {
62 | val count = rdd.count
63 | println("count received " + count)
64 | val sqlContext = SQLContext.getOrCreate(rdd.sparkContext)
65 | import sqlContext.implicits._
66 | import org.apache.spark.sql.functions._
67 |
68 | val sensorDF = rdd.toDF()
69 | // Display the top 20 rows of DataFrame
70 | println("sensor data")
71 | sensorDF.show()
72 | sensorDF.registerTempTable("sensor")
73 | val res = sqlContext.sql("SELECT resid, date, count(resid) as total FROM sensor GROUP BY resid, date")
74 | println("sensor count ")
75 | res.show
76 | val res2 = sqlContext.sql("SELECT resid, date, avg(psi) as avgpsi FROM sensor GROUP BY resid,date")
77 | println("sensor psi average")
78 | res2.show
79 |
80 | }
81 | }
82 | // Start the computation
83 | println("start streaming")
84 | ssc.start()
85 | // Wait for the computation to terminate
86 | ssc.awaitTermination()
87 |
88 | }
89 |
90 | }
--------------------------------------------------------------------------------
/src/main/scala/solution/HBaseReadRowWriteStats.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * This example reads a row of time series sensor data
3 | * calculates the the statistics for the hz data
4 | * and then writes these statistics to the stats column family
5 | *
6 | * you can specify specific columns to return, More info:
7 | * http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html
8 | */
9 |
10 | package solution
11 |
12 | import scala.reflect.runtime.universe
13 |
14 | import org.apache.hadoop.hbase.HBaseConfiguration
15 | import org.apache.hadoop.hbase.client.Put
16 | import org.apache.hadoop.hbase.client.Result
17 | import org.apache.hadoop.hbase.io.ImmutableBytesWritable
18 | import org.apache.hadoop.hbase.mapred.TableOutputFormat
19 | import org.apache.hadoop.hbase.mapreduce.TableInputFormat
20 | import org.apache.hadoop.hbase.util.Bytes
21 | import org.apache.hadoop.mapred.JobConf
22 | import org.apache.spark.SparkConf
23 | import org.apache.spark.SparkContext
24 | import org.apache.spark.rdd.PairRDDFunctions
25 | import org.apache.spark.sql.Row
26 | import org.apache.spark.sql.functions.avg
27 | import org.apache.hadoop.mapreduce.Job
28 | import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
29 | import org.apache.hadoop.fs.Path
30 |
31 | object HBaseReadRowWriteStats {
32 |
33 | case class SensorRow(rowkey: String, hz: Double, disp: Double, flo: Double, sedPPM: Double, psi: Double, chlPPM: Double)
34 |
35 | object SensorRow extends Serializable {
36 | def parseSensorRow(result: Result): SensorRow = {
37 | val rowkey = Bytes.toString(result.getRow())
38 | // remove time from rowKey, stats row key is for day
39 | val p0 = rowkey.split(" ")(0)
40 | val p1 = Bytes.toDouble(result.getValue(cfDataBytes, Bytes.toBytes("hz")))
41 | val p2 = Bytes.toDouble(result.getValue(cfDataBytes, Bytes.toBytes("disp")))
42 | val p3 = Bytes.toDouble(result.getValue(cfDataBytes, Bytes.toBytes("flo")))
43 | val p4 = Bytes.toDouble(result.getValue(cfDataBytes, Bytes.toBytes("sedPPM")))
44 | val p5 = Bytes.toDouble(result.getValue(cfDataBytes, Bytes.toBytes("psi")))
45 | val p6 = Bytes.toDouble(result.getValue(cfDataBytes, Bytes.toBytes("chlPPM")))
46 | SensorRow(p0, p1, p2, p3, p4, p5, p6)
47 | }
48 | }
49 |
50 | case class SensorStatsRow(rowkey: String,
51 | maxhz: Double, minhz: Double, avghz: Double,
52 | maxdisp: Double, mindisp: Double, avgdisp: Double,
53 | maxflo: Double, minflo: Double, avgflo: Double,
54 | maxsedPPM: Double, minsedPPM: Double, avgsedPPM: Double,
55 | maxpsi: Double, minpsi: Double, avgpsi: Double,
56 | maxchlPPM: Double, minchlPPM: Double, avgchlPPM: Double)
57 |
58 | object SensorStatsRow {
59 | def convertToPutStats(row: SensorStatsRow): (ImmutableBytesWritable, Put) = {
60 | val p = new Put(Bytes.toBytes(row.rowkey))
61 | // add columns with data values to put
62 | p.addColumn(cfStatsBytes, Bytes.toBytes("hzmax"), Bytes.toBytes(row.maxhz))
63 | p.addColumn(cfStatsBytes, Bytes.toBytes("hzmin"), Bytes.toBytes(row.minhz))
64 | p.addColumn(cfStatsBytes, Bytes.toBytes("hzavg"), Bytes.toBytes(row.avghz))
65 | p.addColumn(cfStatsBytes, Bytes.toBytes("dispmax"), Bytes.toBytes(row.maxdisp))
66 | p.addColumn(cfStatsBytes, Bytes.toBytes("dispmin"), Bytes.toBytes(row.mindisp))
67 | p.addColumn(cfStatsBytes, Bytes.toBytes("dispavg"), Bytes.toBytes(row.avgdisp))
68 | p.addColumn(cfStatsBytes, Bytes.toBytes("flomax"), Bytes.toBytes(row.maxflo))
69 | p.addColumn(cfStatsBytes, Bytes.toBytes("flomin"), Bytes.toBytes(row.minflo))
70 | p.addColumn(cfStatsBytes, Bytes.toBytes("floavg"), Bytes.toBytes(row.avgflo))
71 | p.addColumn(cfStatsBytes, Bytes.toBytes("sedPPMmax"), Bytes.toBytes(row.maxsedPPM))
72 | p.addColumn(cfStatsBytes, Bytes.toBytes("sedPPMmin"), Bytes.toBytes(row.minsedPPM))
73 | p.addColumn(cfStatsBytes, Bytes.toBytes("sedPPMavg"), Bytes.toBytes(row.avgsedPPM))
74 | p.addColumn(cfStatsBytes, Bytes.toBytes("psimax"), Bytes.toBytes(row.maxpsi))
75 | p.addColumn(cfStatsBytes, Bytes.toBytes("psimin"), Bytes.toBytes(row.minpsi))
76 | p.addColumn(cfStatsBytes, Bytes.toBytes("psiavg"), Bytes.toBytes(row.avgpsi))
77 | p.addColumn(cfStatsBytes, Bytes.toBytes("chlPPMmax"), Bytes.toBytes(row.maxchlPPM))
78 | p.addColumn(cfStatsBytes, Bytes.toBytes("chlPPMmin"), Bytes.toBytes(row.minchlPPM))
79 | p.addColumn(cfStatsBytes, Bytes.toBytes("chlPPMavg"), Bytes.toBytes(row.avgchlPPM))
80 | (new ImmutableBytesWritable, p)
81 | }
82 | }
83 |
84 | final val tableName = "/user/user01/sensor"
85 | final val cfData = "data"
86 | final val cfDataBytes = Bytes.toBytes(cfData)
87 | final val cfStats = "stats"
88 | final val cfStatsBytes = Bytes.toBytes(cfStats)
89 |
90 | def main(args: Array[String]) {
91 | val sparkConf = new SparkConf().setAppName("HBaseTest")
92 | val sc = new SparkContext(sparkConf)
93 | val sqlContext = new org.apache.spark.sql.SQLContext(sc)
94 | import sqlContext.implicits._
95 |
96 | val conf = HBaseConfiguration.create()
97 |
98 | conf.set(TableInputFormat.INPUT_TABLE, tableName)
99 | // scan data column family
100 | conf.set(TableInputFormat.SCAN_COLUMNS, "data")
101 |
102 | // Load an RDD of rowkey, result(ImmutableBytesWritable, Result) tuples from the table
103 | val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat],
104 | classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable],
105 | classOf[org.apache.hadoop.hbase.client.Result])
106 |
107 | hBaseRDD.count()
108 |
109 | // transform (ImmutableBytesWritable, Result) tuples into an RDD of Results
110 | val resultRDD = hBaseRDD.map(tuple => tuple._2)
111 | resultRDD.count()
112 | // transform RDD of Results into an RDD of SensorRow objects
113 | val sensorRDD = resultRDD.map(SensorRow.parseSensorRow)
114 | // change RDD of SensorRow objects to a DataFrame
115 | val sensorDF = sensorRDD.toDF()
116 | // Return the schema of this DataFrame
117 | sensorDF.printSchema()
118 | // Display the top 20 rows of DataFrame
119 | sensorDF.show()
120 | // group by the rowkey (sensorid_date) get average psi
121 | sensorDF.groupBy("rowkey").agg(avg(sensorDF("psi"))).take(5).foreach(println)
122 | // register the DataFrame as a temp table
123 | sensorDF.registerTempTable("SensorRow")
124 |
125 | // group by the rowkey (sensorid_date) get average, max , min for all columns
126 | val sensorStatDF = sqlContext.sql("SELECT rowkey,MAX(hz) as maxhz, min(hz) as minhz, avg(hz) as avghz, MAX(disp) as maxdisp, min(disp) as mindisp, avg(disp) as avgdisp, MAX(flo) as maxflo, min(flo) as minflo, avg(flo) as avgflo,MAX(sedPPM) as maxsedPPM, min(sedPPM) as minsedPPM, avg(sedPPM) as avgsedPPM, MAX(psi) as maxpsi, min(psi) as minpsi, avg(psi) as avgpsi,MAX(chlPPM) as maxchlPPM, min(chlPPM) as minchlPPM, avg(chlPPM) as avgchlPPM FROM SensorRow GROUP BY rowkey")
127 | sensorStatDF.printSchema()
128 | sensorStatDF.take(5).foreach(println)
129 |
130 | // map the query result row to the SensorStatsRow object
131 | val sensorStatsRowRDD = sensorStatDF.map {
132 | case Row(rowkey: String,
133 | maxhz: Double, minhz: Double, avghz: Double, maxdisp: Double, mindisp: Double, avgdisp: Double,
134 | maxflo: Double, minflo: Double, avgflo: Double, maxsedPPM: Double, minsedPPM: Double, avgsedPPM: Double,
135 | maxpsi: Double, minpsi: Double, avgpsi: Double, maxchlPPM: Double, minchlPPM: Double, avgchlPPM: Double) =>
136 | SensorStatsRow(rowkey: String,
137 | maxhz: Double, minhz: Double, avghz: Double, maxdisp: Double, mindisp: Double, avgdisp: Double,
138 | maxflo: Double, minflo: Double, avgflo: Double, maxsedPPM: Double, minsedPPM: Double, avgsedPPM: Double,
139 | maxpsi: Double, minpsi: Double, avgpsi: Double, maxchlPPM: Double, minchlPPM: Double, avgchlPPM: Double)
140 | }
141 |
142 | sensorStatsRowRDD.take(5).foreach(println)
143 |
144 | // set JobConfiguration variables for writing to HBase
145 | val jobConfig: JobConf = new JobConf(conf, this.getClass)
146 | jobConfig.set("mapreduce.output.fileoutputformat.outputdir", "/user/user01/out")
147 | // set the HBase output table
148 | jobConfig.setOutputFormat(classOf[TableOutputFormat])
149 | jobConfig.set(TableOutputFormat.OUTPUT_TABLE, tableName)
150 | // convert the SensorStatsRow objects into HBase put objects and write to HBase
151 | sensorStatsRowRDD.map {
152 | case sensorStatsRow => SensorStatsRow.convertToPutStats(sensorStatsRow)
153 | }.saveAsHadoopDataset(jobConfig)
154 | }
155 |
156 | }
157 |
--------------------------------------------------------------------------------
/src/main/scala/solution/HBaseSensorStream.scala:
--------------------------------------------------------------------------------
1 | package solution
2 |
3 | import org.apache.hadoop.hbase.HBaseConfiguration
4 | import org.apache.hadoop.hbase.client.Put
5 | import org.apache.hadoop.hbase.io.ImmutableBytesWritable
6 | import org.apache.hadoop.hbase.mapred.TableOutputFormat
7 | import org.apache.hadoop.hbase.util.Bytes
8 | import org.apache.hadoop.mapred.JobConf
9 | import org.apache.kafka.clients.consumer.ConsumerConfig
10 | import org.apache.spark.SparkConf
11 |
12 | import org.apache.spark.SparkContext
13 | import org.apache.spark.streaming.Seconds
14 | import org.apache.spark.streaming.StreamingContext
15 | import org.apache.spark.streaming.kafka.v09.KafkaUtils
16 |
17 | object HBaseSensorStream extends Serializable {
18 | final val tableName = "/user/user01/sensor"
19 | final val cfDataBytes = Bytes.toBytes("data")
20 | final val cfAlertBytes = Bytes.toBytes("alert")
21 | final val colHzBytes = Bytes.toBytes("hz")
22 | final val colDispBytes = Bytes.toBytes("disp")
23 | final val colFloBytes = Bytes.toBytes("flo")
24 | final val colSedBytes = Bytes.toBytes("sedPPM")
25 | final val colPsiBytes = Bytes.toBytes("psi")
26 | final val colChlBytes = Bytes.toBytes("chlPPM")
27 |
28 | // schema for sensor data
29 | case class Sensor(resid: String, date: String, time: String, hz: Double, disp: Double, flo: Double, sedPPM: Double, psi: Double, chlPPM: Double) extends Serializable
30 |
31 | object Sensor extends Serializable {
32 | // function to parse line of sensor data into Sensor class
33 | def parseSensor(str: String): Sensor = {
34 | val p = str.split(",")
35 | Sensor(p(0), p(1), p(2), p(3).toDouble, p(4).toDouble, p(5).toDouble, p(6).toDouble, p(7).toDouble, p(8).toDouble)
36 | }
37 | // Convert a row of sensor object data to an HBase put object
38 | def convertToPut(sensor: Sensor): (ImmutableBytesWritable, Put) = {
39 | val dateTime = sensor.date + " " + sensor.time
40 | // create a composite row key: sensorid_date time
41 | val rowkey = sensor.resid + "_" + dateTime
42 | val put = new Put(Bytes.toBytes(rowkey))
43 | // add to column family data, column data values to put object
44 | put.addColumn(cfDataBytes, colHzBytes, Bytes.toBytes(sensor.hz))
45 | put.addColumn(cfDataBytes, colDispBytes, Bytes.toBytes(sensor.disp))
46 | put.addColumn(cfDataBytes, colFloBytes, Bytes.toBytes(sensor.flo))
47 | put.addColumn(cfDataBytes, colSedBytes, Bytes.toBytes(sensor.sedPPM))
48 | put.addColumn(cfDataBytes, colPsiBytes, Bytes.toBytes(sensor.psi))
49 | put.addColumn(cfDataBytes, colChlBytes, Bytes.toBytes(sensor.chlPPM))
50 | return (new ImmutableBytesWritable(Bytes.toBytes(rowkey)), put)
51 | }
52 | // convert psi alert to an HBase put object
53 | def convertToPutAlert(sensor: Sensor): (ImmutableBytesWritable, Put) = {
54 | val dateTime = sensor.date + " " + sensor.time
55 | // create a composite row key: sensorid_date time
56 | val key = sensor.resid + "_" + dateTime
57 | val p = new Put(Bytes.toBytes(key))
58 | // add to column family alert, column psi data value to put object
59 | p.addColumn(cfAlertBytes, colPsiBytes, Bytes.toBytes(sensor.psi))
60 | return (new ImmutableBytesWritable(Bytes.toBytes(key)), p)
61 | }
62 | }
63 |
64 | def main(args: Array[String]): Unit = {
65 |
66 | val brokers = "maprdemo:9092" // not needed for MapR Streams, needed for Kafka
67 | val groupId = "testgroup"
68 | val offsetReset = "earliest"
69 |
70 | val pollTimeout = "1000"
71 | val topics = "/user/user01/pump:sensor"
72 |
73 | // set up HBase Table configuration
74 | val conf = HBaseConfiguration.create()
75 | conf.set(TableOutputFormat.OUTPUT_TABLE, tableName)
76 | val jobConfig: JobConf = new JobConf(conf, this.getClass)
77 | jobConfig.set("mapreduce.output.fileoutputformat.outputdir", "/user/user01/out")
78 | jobConfig.setOutputFormat(classOf[TableOutputFormat])
79 | jobConfig.set(TableOutputFormat.OUTPUT_TABLE, tableName)
80 | println("set configuration")
81 | val sparkConf = new SparkConf().setAppName("HBaseSensorStream")
82 | .set("spark.files.overwrite", "true")
83 | val sc = new SparkContext(sparkConf)
84 |
85 | // create a StreamingContext, the main entry point for all streaming functionality
86 | val ssc = new StreamingContext(sc, Seconds(2))
87 |
88 | // Create direct kafka stream with brokers and topics
89 | val topicsSet = topics.split(",").toSet
90 | val kafkaParams = Map[String, String](
91 | ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> brokers,
92 | ConsumerConfig.GROUP_ID_CONFIG -> groupId,
93 | ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG ->
94 | "org.apache.kafka.common.serialization.StringDeserializer",
95 | ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG ->
96 | "org.apache.kafka.common.serialization.StringDeserializer",
97 | ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> offsetReset,
98 | ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG -> "false",
99 | "spark.kafka.poll.time" -> pollTimeout
100 | )
101 |
102 | // parse the lines of data into sensor objects
103 | val messages = KafkaUtils.createDirectStream[String, String](ssc, kafkaParams, topicsSet)
104 |
105 | val sensorDStream = messages.map(_._2).map(Sensor.parseSensor)
106 |
107 | sensorDStream.print()
108 |
109 | sensorDStream.foreachRDD { rdd =>
110 | // filter sensor data for low psi
111 | val alertRDD = rdd.filter(sensor => sensor.psi < 5.0)
112 | alertRDD.take(1).foreach(println)
113 | // convert sensor data to put object and write to HBase table column family data
114 | rdd.map(Sensor.convertToPut).
115 | saveAsHadoopDataset(jobConfig)
116 | // convert alert data to put object and write to HBase table column family alert
117 | alertRDD.map(Sensor.convertToPutAlert).
118 | saveAsHadoopDataset(jobConfig)
119 | }
120 | // Start the computation
121 | ssc.start()
122 | println("start streaming")
123 | // Wait for the computation to terminate
124 | ssc.awaitTermination()
125 |
126 | }
127 |
128 | }
--------------------------------------------------------------------------------
/src/main/scala/solution/SensorStreamConsumer.scala:
--------------------------------------------------------------------------------
1 | package solution
2 |
3 | import org.apache.kafka.clients.consumer.ConsumerConfig
4 | import org.apache.kafka.common.serialization.StringDeserializer
5 | import org.apache.spark.{ SparkConf, SparkContext }
6 | import org.apache.spark.SparkContext._
7 | import org.apache.spark.streaming._
8 | import org.apache.spark.streaming.dstream.{ DStream, InputDStream }
9 | import org.apache.spark.streaming.kafka.v09.KafkaUtils
10 | import org.apache.spark.streaming.{ Seconds, StreamingContext }
11 | import org.apache.spark.sql.functions.avg
12 | import org.apache.spark.sql.SQLContext
13 |
14 | object SensorStreamConsumer extends Serializable {
15 |
16 | // schema for sensor data
17 | case class Sensor(resid: String, date: String, time: String, hz: Double, disp: Double, flo: Double, sedPPM: Double, psi: Double, chlPPM: Double) extends Serializable
18 |
19 | // function to parse line of sensor data into Sensor class
20 | def parseSensor(str: String): Sensor = {
21 | val p = str.split(",")
22 | Sensor(p(0), p(1), p(2), p(3).toDouble, p(4).toDouble, p(5).toDouble, p(6).toDouble, p(7).toDouble, p(8).toDouble)
23 | }
24 | val timeout = 10 // Terminate after N seconds
25 | val batchSeconds = 2 // Size of batch intervals
26 |
27 | def main(args: Array[String]): Unit = {
28 |
29 | val brokers = "maprdemo:9092" // not needed for MapR Streams, needed for Kafka
30 | val groupId = "testgroup"
31 | val offsetReset = "earliest"
32 | val batchInterval = "2"
33 | val pollTimeout = "1000"
34 | val topics = "/user/user01/pump:sensor"
35 |
36 | val sparkConf = new SparkConf().setAppName("SensorStream")
37 |
38 | val ssc = new StreamingContext(sparkConf, Seconds(batchInterval.toInt))
39 |
40 | // Create direct kafka stream with brokers and topics
41 | val topicsSet = topics.split(",").toSet
42 | val kafkaParams = Map[String, String](
43 | ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> brokers,
44 | ConsumerConfig.GROUP_ID_CONFIG -> groupId,
45 | ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG ->
46 | "org.apache.kafka.common.serialization.StringDeserializer",
47 | ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG ->
48 | "org.apache.kafka.common.serialization.StringDeserializer",
49 | ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> offsetReset,
50 | ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG -> "false",
51 | "spark.kafka.poll.time" -> pollTimeout
52 | )
53 |
54 | val messages = KafkaUtils.createDirectStream[String, String](ssc, kafkaParams, topicsSet)
55 |
56 | val sensorDStream = messages.map(_._2).map(parseSensor)
57 |
58 | sensorDStream.foreachRDD { rdd =>
59 |
60 | // There exists at least one element in RDD
61 | if (!rdd.isEmpty) {
62 | val count = rdd.count
63 | println("count received " + count)
64 | val sqlContext = SQLContext.getOrCreate(rdd.sparkContext)
65 | import sqlContext.implicits._
66 | import org.apache.spark.sql.functions._
67 |
68 | val sensorDF = rdd.toDF()
69 | // Display the top 20 rows of DataFrame
70 | println("sensor data")
71 | sensorDF.show()
72 | sensorDF.registerTempTable("sensor")
73 | val res = sqlContext.sql("SELECT resid, date, count(resid) as total FROM sensor GROUP BY resid, date")
74 | println("sensor count ")
75 | res.show
76 | val res2 = sqlContext.sql("SELECT resid, date, avg(psi) as avgpsi FROM sensor GROUP BY resid,date")
77 | println("sensor psi average")
78 | res2.show
79 |
80 | }
81 | }
82 | // Start the computation
83 | println("start streaming")
84 | ssc.start()
85 | // Wait for the computation to terminate
86 | ssc.awaitTermination()
87 |
88 | }
89 |
90 | }
--------------------------------------------------------------------------------