-
Notifications
You must be signed in to change notification settings - Fork 28.1k
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
[SPARK-5155] [PySpark] [Streaming] Mqtt streaming support in Python #4229
Changes from 17 commits
b7d42ff
3aa7fff
b34c3c1
3f4df12
ee387ae
795ec27
a11968b
9767d82
a5a8f9f
e1ee016
1f0cfe9
80474d1
97244ec
87fc677
a6747cb
d07f454
b90b709
126608a
734db99
5f8a1d4
478f844
47278c5
935615c
abf5f18
03f3e88
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 |
---|---|---|
@@ -0,0 +1,57 @@ | ||
# | ||
# 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. | ||
# | ||
|
||
""" | ||
A sample wordcount with MqttStream stream | ||
Usage: mqtt_wordcount.py <broker url> <topic> | ||
To work with Mqtt, Mqtt Message broker/server required. | ||
Mosquitto (http://mosquitto.org/) is an open source Mqtt Broker | ||
In ubuntu mosquitto can be installed using the command `$ sudo apt-get install mosquitto` | ||
Run Mqtt publisher as | ||
`$ bin/run-example \ | ||
org.apache.spark.examples.streaming.MQTTPublisher tcp://localhost:1883 foo` | ||
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. 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 that's fine. We can also add something to highlight this |
||
and then run the example as | ||
`$ bin/spark-submit --jars external/mqtt-assembly/target/scala-*/\ | ||
spark-streaming-mqtt-assembly-*.jar examples/src/main/python/streaming/mqtt_wordcount.py \ | ||
tcp://localhost:1883 foo` | ||
""" | ||
|
||
import sys | ||
|
||
from pyspark import SparkContext | ||
from pyspark.streaming import StreamingContext | ||
from pyspark.streaming.mqtt import MQTTUtils | ||
|
||
if __name__ == "__main__": | ||
if len(sys.argv) != 3: | ||
print >> sys.stderr, "Usage: mqtt_wordcount.py <broker url> <topic>" | ||
exit(-1) | ||
|
||
sc = SparkContext(appName="PythonStreamingMQTTWordCount") | ||
ssc = StreamingContext(sc, 1) | ||
|
||
brokerUrl = sys.argv[1] | ||
topic = sys.argv[2] | ||
|
||
lines = MQTTUtils.createStream(ssc, brokerUrl, topic) | ||
counts = lines.flatMap(lambda line: line.split(" ")) \ | ||
.map(lambda word: (word, 1)) \ | ||
.reduceByKey(lambda a, b: a+b) | ||
counts.pprint() | ||
|
||
ssc.start() | ||
ssc.awaitTermination() |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,102 @@ | ||
<?xml version="1.0" encoding="UTF-8"?> | ||
<!-- | ||
~ 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. | ||
--> | ||
|
||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> | ||
<modelVersion>4.0.0</modelVersion> | ||
<parent> | ||
<groupId>org.apache.spark</groupId> | ||
<artifactId>spark-parent_2.10</artifactId> | ||
<version>1.5.0-SNAPSHOT</version> | ||
<relativePath>../../pom.xml</relativePath> | ||
</parent> | ||
|
||
<groupId>org.apache.spark</groupId> | ||
<artifactId>spark-streaming-mqtt-assembly_2.10</artifactId> | ||
<packaging>jar</packaging> | ||
<name>Spark Project External MQTT Assembly</name> | ||
<url>http://spark.apache.org/</url> | ||
|
||
<properties> | ||
<sbt.project.name>streaming-mqtt-assembly</sbt.project.name> | ||
</properties> | ||
|
||
<dependencies> | ||
<dependency> | ||
<groupId>org.apache.spark</groupId> | ||
<artifactId>spark-streaming-mqtt_${scala.binary.version}</artifactId> | ||
<version>${project.version}</version> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.spark</groupId> | ||
<artifactId>spark-streaming_${scala.binary.version}</artifactId> | ||
<version>${project.version}</version> | ||
<scope>provided</scope> | ||
</dependency> | ||
</dependencies> | ||
|
||
<build> | ||
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory> | ||
<testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory> | ||
<plugins> | ||
<plugin> | ||
<groupId>org.apache.maven.plugins</groupId> | ||
<artifactId>maven-shade-plugin</artifactId> | ||
<configuration> | ||
<shadedArtifactAttached>false</shadedArtifactAttached> | ||
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. Could you add a line like https://github.com/apache/spark/blob/master/external/kafka-assembly/pom.xml#L61 ? See #5632 for the issue. |
||
<outputFile>${project.build.directory}/scala-${scala.binary.version}/spark-streaming-mqtt-assembly-${project.version}.jar</outputFile> | ||
<artifactSet> | ||
<includes> | ||
<include>*:*</include> | ||
</includes> | ||
</artifactSet> | ||
<filters> | ||
<filter> | ||
<artifact>*:*</artifact> | ||
<excludes> | ||
<exclude>META-INF/*.SF</exclude> | ||
<exclude>META-INF/*.DSA</exclude> | ||
<exclude>META-INF/*.RSA</exclude> | ||
</excludes> | ||
</filter> | ||
</filters> | ||
</configuration> | ||
<executions> | ||
<execution> | ||
<phase>package</phase> | ||
<goals> | ||
<goal>shade</goal> | ||
</goals> | ||
<configuration> | ||
<transformers> | ||
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/> | ||
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer"> | ||
<resource>reference.conf</resource> | ||
</transformer> | ||
<transformer implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer"> | ||
<resource>log4j.properties</resource> | ||
</transformer> | ||
<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer"/> | ||
<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer"/> | ||
</transformers> | ||
</configuration> | ||
</execution> | ||
</executions> | ||
</plugin> | ||
</plugins> | ||
</build> | ||
</project> |
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 should be ran inside of mosquitto, could you mention that?
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.
Can we modify like this ?
Publisher publishes data to mosquitto broker set in local host. Run Mqtt publisher as
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.
bin/run-example is also inside spark, I misunderstood that, sorry.
never-mind.
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.
Ya it is actually running the Scala publisher inside the Scala streaming word count example