Skip to content

Commit

Permalink
[BAHIR-100] Enhance MQTT connector to support byte arrays
Browse files Browse the repository at this point in the history
Closes apache#47
  • Loading branch information
davidrosenstark authored and lresende committed Jul 19, 2017
1 parent dca8d4c commit e3d9e69
Show file tree
Hide file tree
Showing 7 changed files with 477 additions and 84 deletions.
3 changes: 3 additions & 0 deletions streaming-mqtt/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -52,12 +52,14 @@ this actor can be configured to handle failures, etc.

val lines = MQTTUtils.createStream(ssc, brokerUrl, topic)
val lines = MQTTUtils.createPairedStream(ssc, brokerUrl, topic)
val lines = MQTTUtils.createPairedByteArrayStreamStream(ssc, brokerUrl, topic)

Additional mqtt connection options can be provided:

```Scala
val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, storageLevel, clientId, username, password, cleanSession, qos, connectionTimeout, keepAliveInterval, mqttVersion)
val lines = MQTTUtils.createPairedStream(ssc, brokerUrl, topics, storageLevel, clientId, username, password, cleanSession, qos, connectionTimeout, keepAliveInterval, mqttVersion)
val lines = MQTTUtils.createPairedByteArrayStream(ssc, brokerUrl, topics, storageLevel, clientId, username, password, cleanSession, qos, connectionTimeout, keepAliveInterval, mqttVersion)
```

### Java API
Expand All @@ -67,5 +69,6 @@ this actor can be configured to handle failures, etc.

JavaDStream<String> lines = MQTTUtils.createStream(jssc, brokerUrl, topic);
JavaReceiverInputDStream<Tuple2<String, String>> lines = MQTTUtils.createPairedStream(jssc, brokerUrl, topics);
JavaReceiverInputDStream<Tuple2<String, String>> lines = MQTTUtils.createPairedByteArrayStream(jssc, brokerUrl, topics);

See end-to-end examples at [MQTT Examples](https://github.com/apache/bahir/tree/master/streaming-mqtt/examples)
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import org.apache.spark.streaming.receiver.Receiver
/**
* Input stream that subscribe messages from a Mqtt Broker.
* Uses eclipse paho as MqttClient http://www.eclipse.org/paho/
* @param _ssc Spark Streaming StreamingContext
* @param brokerUrl Url of remote mqtt publisher
* @param topic topic name to subscribe to
* @param storageLevel RDD storage level.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,144 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.streaming.mqtt

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.StreamingContext
import org.apache.spark.streaming.dstream._
import org.apache.spark.streaming.receiver.Receiver

/**
* Input stream that subscribe messages from a Mqtt Broker.
* Uses eclipse paho as MqttClient http://www.eclipse.org/paho/
* @param _ssc: Spark Streaming StreamingContext,
* @param brokerUrl Url of remote mqtt publisher
* @param topics topic name Array to subscribe to
* @param storageLevel RDD storage level.
* @param clientId ClientId to use for the mqtt connection
* @param username Username for authentication to the mqtt publisher
* @param password Password for authentication to the mqtt publisher
* @param cleanSession Sets the mqtt cleanSession parameter
* @param qos Quality of service to use for the topic subscription
* @param connectionTimeout Connection timeout for the mqtt connection
* @param keepAliveInterval Keepalive interal for the mqtt connection
* @param mqttVersion Version to use for the mqtt connection
*/
private[streaming] class MQTTPairedByteArrayInputDStream(
_ssc: StreamingContext,
brokerUrl: String,
topics: Array[String],
storageLevel: StorageLevel,
clientId: Option[String] = None,
username: Option[String] = None,
password: Option[String] = None,
cleanSession: Option[Boolean] = None,
qos: Option[Int] = None,
connectionTimeout: Option[Int] = None,
keepAliveInterval: Option[Int] = None,
mqttVersion: Option[Int] = None) extends ReceiverInputDStream[(String, Array[Byte])](_ssc) {

private[streaming] override def name: String = s"MQTT stream [$id]"

def getReceiver(): Receiver[(String, Array[Byte])] = {
new MQTTByteArrayPairReceiver(brokerUrl, topics, storageLevel, clientId, username,
password, cleanSession, qos, connectionTimeout, keepAliveInterval, mqttVersion)
}
}

private[streaming] class MQTTByteArrayPairReceiver(
brokerUrl: String,
topics: Array[String],
storageLevel: StorageLevel,
clientId: Option[String],
username: Option[String],
password: Option[String],
cleanSession: Option[Boolean],
qos: Option[Int],
connectionTimeout: Option[Int],
keepAliveInterval: Option[Int],
mqttVersion: Option[Int]) extends Receiver[(String, Array[Byte])](storageLevel) {

def onStop() {

}

def onStart() {

// Set up persistence for messages
val persistence = new MemoryPersistence()

// Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance
val client = new MqttClient(brokerUrl, clientId.getOrElse(MqttClient.generateClientId()),
persistence)

// Initialize mqtt parameters
val mqttConnectionOptions = new MqttConnectOptions()
if (username.isDefined && password.isDefined) {
mqttConnectionOptions.setUserName(username.get)
mqttConnectionOptions.setPassword(password.get.toCharArray)
}
mqttConnectionOptions.setCleanSession(cleanSession.getOrElse(true))
if (connectionTimeout.isDefined) {
mqttConnectionOptions.setConnectionTimeout(connectionTimeout.get)
}
if (keepAliveInterval.isDefined) {
mqttConnectionOptions.setKeepAliveInterval(keepAliveInterval.get)
}
if (mqttVersion.isDefined) {
mqttConnectionOptions.setMqttVersion(mqttVersion.get)
}

// Callback automatically triggers as and when new message arrives on specified topic
val callback = new MqttCallback() {

// Handles Mqtt message
override def messageArrived(topic: String, message: MqttMessage) {
store((topic, message.getPayload()))
}

override def deliveryComplete(token: IMqttDeliveryToken) {
}

override def connectionLost(cause: Throwable) {
restart("Connection lost ", cause)
}
}

// Set up callback for MqttClient. This needs to happen before
// connecting or subscribing, otherwise messages may be lost
client.setCallback(callback)

// Connect to MqttBroker
client.connect(mqttConnectionOptions)

// Subscribe to Mqtt topic
var i = 0
val qosArray = Array.ofDim[Int](topics.length)
for (i <- qosArray.indices) {
qosArray(i) = qos.getOrElse(1)
}
client.subscribe(topics, qosArray)

}
}



Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import org.apache.spark.streaming.receiver.Receiver
/**
* Input stream that subscribe messages from a Mqtt Broker.
* Uses eclipse paho as MqttClient http://www.eclipse.org/paho/
* @param _ssc Spark Streaming StreamingContext
* @param brokerUrl Url of remote mqtt publisher
* @param topics topic name Array to subscribe to
* @param storageLevel RDD storage level.
Expand Down
Loading

0 comments on commit e3d9e69

Please sign in to comment.