Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3844#discussion_r22429050
  
    --- Diff: 
external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
 ---
    @@ -17,31 +17,114 @@
     
     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 10)
    +          msgTopic.publish(message)
    +      }
    +    } catch {
    +      case e: MqttException => println("Exception Caught: " + e)
    --- End diff --
    
    Why can there be an exception? And if there is an exception, why is it 
being ignored? Printing and not doing anything is essentially ignoring if the 
unit test passes without successfully testing anything. If there is a slightly 
chance that this publishing may fail, please add sufficient retries to make it 
pass reliably.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to