├── .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 | } --------------------------------------------------------------------------------