aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/streaming/tests.py
diff options
context:
space:
mode:
Diffstat (limited to 'python/pyspark/streaming/tests.py')
-rw-r--r--python/pyspark/streaming/tests.py106
1 files changed, 104 insertions, 2 deletions
diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py
index 5cd544b214..66ae3345f4 100644
--- a/python/pyspark/streaming/tests.py
+++ b/python/pyspark/streaming/tests.py
@@ -40,6 +40,7 @@ from pyspark.storagelevel import StorageLevel
from pyspark.streaming.context import StreamingContext
from pyspark.streaming.kafka import Broker, KafkaUtils, OffsetRange, TopicAndPartition
from pyspark.streaming.flume import FlumeUtils
+from pyspark.streaming.mqtt import MQTTUtils
from pyspark.streaming.kinesis import KinesisUtils, InitialPositionInStream
@@ -893,6 +894,68 @@ class FlumePollingStreamTests(PySparkStreamingTestCase):
self._testMultipleTimes(self._testFlumePollingMultipleHosts)
+class MQTTStreamTests(PySparkStreamingTestCase):
+ timeout = 20 # seconds
+ duration = 1
+
+ def setUp(self):
+ super(MQTTStreamTests, self).setUp()
+
+ MQTTTestUtilsClz = self.ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \
+ .loadClass("org.apache.spark.streaming.mqtt.MQTTTestUtils")
+ self._MQTTTestUtils = MQTTTestUtilsClz.newInstance()
+ self._MQTTTestUtils.setup()
+
+ def tearDown(self):
+ if self._MQTTTestUtils is not None:
+ self._MQTTTestUtils.teardown()
+ self._MQTTTestUtils = None
+
+ super(MQTTStreamTests, self).tearDown()
+
+ def _randomTopic(self):
+ return "topic-%d" % random.randint(0, 10000)
+
+ def _startContext(self, topic):
+ # Start the StreamingContext and also collect the result
+ stream = MQTTUtils.createStream(self.ssc, "tcp://" + self._MQTTTestUtils.brokerUri(), topic)
+ result = []
+
+ def getOutput(_, rdd):
+ for data in rdd.collect():
+ result.append(data)
+
+ stream.foreachRDD(getOutput)
+ self.ssc.start()
+ return result
+
+ def test_mqtt_stream(self):
+ """Test the Python MQTT stream API."""
+ sendData = "MQTT demo for spark streaming"
+ topic = self._randomTopic()
+ result = self._startContext(topic)
+
+ def retry():
+ self._MQTTTestUtils.publishData(topic, sendData)
+ # Because "publishData" sends duplicate messages, here we should use > 0
+ self.assertTrue(len(result) > 0)
+ self.assertEqual(sendData, result[0])
+
+ # Retry it because we don't know when the receiver will start.
+ self._retry_or_timeout(retry)
+
+ def _retry_or_timeout(self, test_func):
+ start_time = time.time()
+ while True:
+ try:
+ test_func()
+ break
+ except:
+ if time.time() - start_time > self.timeout:
+ raise
+ time.sleep(0.01)
+
+
class KinesisStreamTests(PySparkStreamingTestCase):
def test_kinesis_stream_api(self):
@@ -985,7 +1048,42 @@ def search_flume_assembly_jar():
"'build/mvn package' before running this test")
elif len(jars) > 1:
raise Exception(("Found multiple Spark Streaming Flume assembly JARs in %s; please "
- "remove all but one") % flume_assembly_dir)
+ "remove all but one") % flume_assembly_dir)
+ else:
+ return jars[0]
+
+
+def search_mqtt_assembly_jar():
+ SPARK_HOME = os.environ["SPARK_HOME"]
+ mqtt_assembly_dir = os.path.join(SPARK_HOME, "external/mqtt-assembly")
+ jars = glob.glob(
+ os.path.join(mqtt_assembly_dir, "target/scala-*/spark-streaming-mqtt-assembly-*.jar"))
+ if not jars:
+ raise Exception(
+ ("Failed to find Spark Streaming MQTT assembly jar in %s. " % mqtt_assembly_dir) +
+ "You need to build Spark with "
+ "'build/sbt assembly/assembly streaming-mqtt-assembly/assembly' or "
+ "'build/mvn package' before running this test")
+ elif len(jars) > 1:
+ raise Exception(("Found multiple Spark Streaming MQTT assembly JARs in %s; please "
+ "remove all but one") % mqtt_assembly_dir)
+ else:
+ return jars[0]
+
+
+def search_mqtt_test_jar():
+ SPARK_HOME = os.environ["SPARK_HOME"]
+ mqtt_test_dir = os.path.join(SPARK_HOME, "external/mqtt")
+ jars = glob.glob(
+ os.path.join(mqtt_test_dir, "target/scala-*/spark-streaming-mqtt-test-*.jar"))
+ if not jars:
+ raise Exception(
+ ("Failed to find Spark Streaming MQTT test jar in %s. " % mqtt_test_dir) +
+ "You need to build Spark with "
+ "'build/sbt assembly/assembly streaming-mqtt/test:assembly'")
+ elif len(jars) > 1:
+ raise Exception(("Found multiple Spark Streaming MQTT test JARs in %s; please "
+ "remove all but one") % mqtt_test_dir)
else:
return jars[0]
@@ -1012,8 +1110,12 @@ def search_kinesis_asl_assembly_jar():
if __name__ == "__main__":
kafka_assembly_jar = search_kafka_assembly_jar()
flume_assembly_jar = search_flume_assembly_jar()
+ mqtt_assembly_jar = search_mqtt_assembly_jar()
+ mqtt_test_jar = search_mqtt_test_jar()
kinesis_asl_assembly_jar = search_kinesis_asl_assembly_jar()
- jars = "%s,%s,%s" % (kafka_assembly_jar, flume_assembly_jar, kinesis_asl_assembly_jar)
+
+ jars = "%s,%s,%s,%s,%s" % (kafka_assembly_jar, flume_assembly_jar, kinesis_asl_assembly_jar,
+ mqtt_assembly_jar, mqtt_test_jar)
os.environ["PYSPARK_SUBMIT_ARGS"] = "--jars %s pyspark-shell" % jars
unittest.main()