Skip to content
6 changes: 6 additions & 0 deletions external/mqtt/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,12 @@
<artifactId>junit-interface</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.activemq</groupId>
<artifactId>activemq-core</artifactId>
<version>5.7.0</version>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,31 +17,111 @@

package org.apache.spark.streaming.mqtt

import org.scalatest.FunSuite
import java.net.{URI, ServerSocket}

import org.apache.spark.streaming.{Seconds, StreamingContext}
import org.apache.activemq.broker.{TransportConnector, BrokerService}
import org.apache.spark.util.Utils
import org.scalatest.{BeforeAndAfter, FunSuite}
import org.scalatest.concurrent.Eventually
import scala.concurrent.duration._
import org.apache.spark.streaming.{Milliseconds, StreamingContext}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.dstream.ReceiverInputDStream
import org.eclipse.paho.client.mqttv3._
import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence

class MQTTStreamSuite extends FunSuite {

val batchDuration = Seconds(1)
class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter {

private val batchDuration = Milliseconds(500)
private val master: String = "local[2]"

private val framework: String = this.getClass.getSimpleName
private val freePort = findFreePort()
private val brokerUri = "//localhost:" + freePort
private val topic = "def"
private var ssc: StreamingContext = _
private val persistenceDir = Utils.createTempDir()
private var broker: BrokerService = _
private var connector: TransportConnector = _

test("mqtt input stream") {
val ssc = new StreamingContext(master, framework, batchDuration)
val brokerUrl = "abc"
val topic = "def"
before {
ssc = new StreamingContext(master, framework, batchDuration)
setupMQTT()
}

// tests the API, does not actually test data receiving
val test1: ReceiverInputDStream[String] = MQTTUtils.createStream(ssc, brokerUrl, topic)
val test2: ReceiverInputDStream[String] =
MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2)
after {
if (ssc != null) {
ssc.stop()
ssc = null
}
Utils.deleteRecursively(persistenceDir)
tearDownMQTT()
}

// TODO: Actually test receiving data
test("mqtt input stream") {
val sendMessage = "MQTT demo for spark streaming"
val receiveStream: ReceiverInputDStream[String] =
MQTTUtils.createStream(ssc, "tcp:" + brokerUri, topic, StorageLevel.MEMORY_ONLY)
var receiveMessage: List[String] = List()
receiveStream.foreachRDD { rdd =>
if (rdd.collect.length > 0) {
receiveMessage = receiveMessage ::: List(rdd.first)
receiveMessage
}
}
ssc.start()
publishData(sendMessage)
eventually(timeout(10000 milliseconds), interval(100 milliseconds)) {
assert(sendMessage.equals(receiveMessage(0)))
}
ssc.stop()
}

private def setupMQTT() {
broker = new BrokerService()
connector = new TransportConnector()
connector.setName("mqtt")
connector.setUri(new URI("mqtt:" + brokerUri))
broker.addConnector(connector)
broker.start()
}

private def tearDownMQTT() {
if (broker != null) {
broker.stop()
broker = null
}
if (connector != null) {
connector.stop()
connector = null
}
}

private def findFreePort(): Int = {
Utils.startServiceOnPort(23456, (trialPort: Int) => {
val socket = new ServerSocket(trialPort)
socket.close()
(null, trialPort)
})._2
}

def publishData(data: String): Unit = {
var client: MqttClient = null
try {
val persistence: MqttClientPersistence = new MqttDefaultFilePersistence(persistenceDir.getAbsolutePath)
client = new MqttClient("tcp:" + brokerUri, MqttClient.generateClientId(), persistence)
client.connect()
if (client.isConnected) {
val msgTopic: MqttTopic = client.getTopic(topic)
val message: MqttMessage = new MqttMessage(data.getBytes("utf-8"))
message.setQos(1)
message.setRetained(true)
for (i <- 0 to 100)
msgTopic.publish(message)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: I missed this in the last pass, but this violates the Scala syntax that we follow. I wont block this PR for this.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you explain what is the correction here. Just to understand what went wrong.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

for (...) {
    msgTopic.publish(message)
}

Such code block should either be in one line or be within braces.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok.. thanks.

}
} finally {
client.disconnect()
client.close()
client = null
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldnt there be a finally to close any running servers (client, etc.)?

}
}