-
Notifications
You must be signed in to change notification settings - Fork 29.1k
[SPARK-5666][streaming][MQTT streaming] some trivial fixes #4178
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
5857989
66919a3
cd57029
838c38e
0cc67bd
22dd7f7
c035bdc
46f9619
ccc0765
bd2cb49
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,8 +17,8 @@ | |
|
|
||
| package org.apache.spark.examples.streaming | ||
|
|
||
| import org.eclipse.paho.client.mqttv3.{MqttClient, MqttClientPersistence, MqttException, MqttMessage, MqttTopic} | ||
| import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence | ||
| import org.eclipse.paho.client.mqttv3._ | ||
| import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence | ||
|
|
||
| import org.apache.spark.storage.StorageLevel | ||
| import org.apache.spark.streaming.{Seconds, StreamingContext} | ||
|
|
@@ -31,8 +31,6 @@ import org.apache.spark.SparkConf | |
| */ | ||
| object MQTTPublisher { | ||
|
|
||
| var client: MqttClient = _ | ||
|
|
||
| def main(args: Array[String]) { | ||
| if (args.length < 2) { | ||
| System.err.println("Usage: MQTTPublisher <MqttBrokerUrl> <topic>") | ||
|
|
@@ -42,25 +40,36 @@ object MQTTPublisher { | |
| StreamingExamples.setStreamingLogLevels() | ||
|
|
||
| val Seq(brokerUrl, topic) = args.toSeq | ||
|
|
||
| var client: MqttClient = null | ||
|
|
||
| try { | ||
| var peristance:MqttClientPersistence =new MqttDefaultFilePersistence("/tmp") | ||
| client = new MqttClient(brokerUrl, MqttClient.generateClientId(), peristance) | ||
| val persistence = new MemoryPersistence() | ||
| client = new MqttClient(brokerUrl, MqttClient.generateClientId(), persistence) | ||
|
|
||
| client.connect() | ||
|
|
||
| val msgtopic = client.getTopic(topic) | ||
| val msgContent = "hello mqtt demo for spark streaming" | ||
| val message = new MqttMessage(msgContent.getBytes("utf-8")) | ||
|
|
||
| while (true) { | ||
| try { | ||
| msgtopic.publish(message) | ||
| println(s"Published data. topic: {msgtopic.getName()}; Message: {message}") | ||
| } catch { | ||
| case e: MqttException if e.getReasonCode == MqttException.REASON_CODE_MAX_INFLIGHT => | ||
| Thread.sleep(10) | ||
| println("Queue is full, wait for to consume data from the message queue") | ||
| } | ||
| } | ||
| } catch { | ||
| case e: MqttException => println("Exception Caught: " + e) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't see the point in swallowing this exception. Why not let it percolate, it will anyway print the stacktrace and stop the thread? |
||
| } finally { | ||
| if (client != null) { | ||
| client.disconnect() | ||
| } | ||
| } | ||
|
|
||
| client.connect() | ||
|
|
||
| val msgtopic: MqttTopic = client.getTopic(topic) | ||
| val msg: String = "hello mqtt demo for spark streaming" | ||
|
|
||
| while (true) { | ||
| val message: MqttMessage = new MqttMessage(String.valueOf(msg).getBytes("utf-8")) | ||
| msgtopic.publish(message) | ||
| println("Published data. topic: " + msgtopic.getName() + " Message: " + message) | ||
| } | ||
| client.disconnect() | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -96,9 +105,9 @@ object MQTTWordCount { | |
| val sparkConf = new SparkConf().setAppName("MQTTWordCount") | ||
| val ssc = new StreamingContext(sparkConf, Seconds(2)) | ||
| val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_ONLY_SER_2) | ||
|
|
||
| val words = lines.flatMap(x => x.toString.split(" ")) | ||
| val words = lines.flatMap(x => x.split(" ")) | ||
| val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) | ||
|
|
||
| wordCounts.print() | ||
| ssc.start() | ||
| ssc.awaitTermination() | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,23 +17,23 @@ | |
|
|
||
| package org.apache.spark.streaming.mqtt | ||
|
|
||
| import java.io.IOException | ||
| import java.util.concurrent.Executors | ||
| import java.util.Properties | ||
|
|
||
| import scala.collection.JavaConversions._ | ||
| import scala.collection.Map | ||
| import scala.collection.mutable.HashMap | ||
| import scala.collection.JavaConversions._ | ||
| import scala.reflect.ClassTag | ||
|
|
||
| import java.util.Properties | ||
| import java.util.concurrent.Executors | ||
| import java.io.IOException | ||
|
|
||
| import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken | ||
| import org.eclipse.paho.client.mqttv3.MqttCallback | ||
| import org.eclipse.paho.client.mqttv3.MqttClient | ||
| import org.eclipse.paho.client.mqttv3.MqttClientPersistence | ||
| import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence | ||
| import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken | ||
| import org.eclipse.paho.client.mqttv3.MqttException | ||
| import org.eclipse.paho.client.mqttv3.MqttMessage | ||
| import org.eclipse.paho.client.mqttv3.MqttTopic | ||
| import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Actually which is the right import style. The below or existing ? import org.eclipse.paho.client.mqttv3.{IMqttDeliveryToken, MqttCallback, MqttClient, MqttClientPersistence, MqttException, MqttMessage, MqttTopic}
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think the multiple import syntax is often used, especially where it's definitely shorter, but, I would not change it just to change it. |
||
|
|
||
| import org.apache.spark.Logging | ||
| import org.apache.spark.storage.StorageLevel | ||
|
|
@@ -88,18 +88,18 @@ class MQTTReceiver( | |
| client.subscribe(topic) | ||
|
|
||
| // Callback automatically triggers as and when new message arrives on specified topic | ||
| val callback: MqttCallback = new MqttCallback() { | ||
| val callback = new MqttCallback() { | ||
|
|
||
| // Handles Mqtt message | ||
| override def messageArrived(arg0: String, arg1: MqttMessage) { | ||
| store(new String(arg1.getPayload(),"utf-8")) | ||
| override def messageArrived(topic: String, message: MqttMessage) { | ||
| store(new String(message.getPayload(),"utf-8")) | ||
| } | ||
|
|
||
| override def deliveryComplete(arg0: IMqttDeliveryToken) { | ||
| override def deliveryComplete(token: IMqttDeliveryToken) { | ||
| } | ||
|
|
||
| override def connectionLost(arg0: Throwable) { | ||
| restart("Connection lost ", arg0) | ||
| override def connectionLost(cause: Throwable) { | ||
| restart("Connection lost ", cause) | ||
| } | ||
| } | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is there any risk here that it will just keep erroring forever? should there be a max number of failures?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this is an example, and it's supposed to run forever, people have to kill the process anyway (with or without errors).
or
should edit this for checking max number of failures ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK, at least log a warning. Or else this could silently spin forever as long as the queue is full. Yes it's an example but still seems funny.
Also, really minor but the
org.apache.spark.streaming.StreamingContextimport you moved inMQTTUtilsis now out of order.