[
https://issues.apache.org/jira/browse/STORM-1406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15085570#comment-15085570
]
ASF GitHub Bot commented on STORM-1406:
---------------------------------------
Github user HeartSaVioR commented on a diff in the pull request:
https://github.com/apache/storm/pull/991#discussion_r48960421
--- Diff:
external/storm-mqtt/core/src/main/java/org/apache/storm/mqtt/spout/MqttSpout.java
---
@@ -0,0 +1,291 @@
+/**
+ * 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.storm.mqtt.spout;
+
+import backtype.storm.Config;
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichSpout;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.mqtt.MqttLogger;
+import org.apache.storm.mqtt.MqttMessageMapper;
+import org.apache.storm.mqtt.common.MqttOptions;
+import org.apache.storm.mqtt.common.MqttUtils;
+import org.apache.storm.mqtt.common.SslUtils;
+import org.apache.storm.mqtt.ssl.KeyStoreLoader;
+import org.fusesource.hawtbuf.Buffer;
+import org.fusesource.hawtbuf.UTF8Buffer;
+import org.fusesource.mqtt.client.Callback;
+import org.fusesource.mqtt.client.CallbackConnection;
+import org.fusesource.mqtt.client.Listener;
+import org.fusesource.mqtt.client.MQTT;
+import org.fusesource.mqtt.client.QoS;
+import org.fusesource.mqtt.client.Topic;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+
+public class MqttSpout implements IRichSpout, Listener {
+ private static final Logger LOG =
LoggerFactory.getLogger(MqttSpout.class);
+
+ private String topologyName;
+
+
+ private CallbackConnection connection;
+
+ protected transient SpoutOutputCollector collector;
+ protected transient TopologyContext context;
+ protected transient LinkedBlockingQueue<AckableMessage> incoming;
+ protected transient HashMap<Long, AckableMessage> pending;
+ private transient Map conf;
+ protected MqttMessageMapper type;
+ protected MqttOptions options;
+ protected KeyStoreLoader keyStoreLoader;
+
+ private boolean mqttConnected = false;
+ private boolean mqttConnectFailed = false;
+
+
+ private Long sequence = Long.MIN_VALUE;
+
+ private Long nextId(){
+ this.sequence++;
+ if(this.sequence == Long.MAX_VALUE){
+ this.sequence = Long.MIN_VALUE;
+ }
+ return this.sequence;
+ }
+
+
+ public MqttSpout(MqttMessageMapper type, MqttOptions options){
+ this(type, options, null);
+ }
+
+ public MqttSpout(MqttMessageMapper type, MqttOptions options,
KeyStoreLoader keyStoreLoader){
+ this.type = type;
+ this.options = options;
+ this.keyStoreLoader = keyStoreLoader;
+ SslUtils.checkSslConfig(this.options.getUrl(),
this.keyStoreLoader);
+ }
+
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ declarer.declare(this.type.outputFields());
+ }
+
+ public Map<String, Object> getComponentConfiguration() {
+ return null;
+ }
+
+ public void open(Map conf, TopologyContext context,
SpoutOutputCollector collector) {
+ this.topologyName = (String)conf.get(Config.TOPOLOGY_NAME);
+
+ this.collector = collector;
+ this.context = context;
+ this.conf = conf;
+
+ this.incoming = new LinkedBlockingQueue<>();
+ this.pending = new HashMap<>();
+
+ try {
+ connectMqtt();
+ } catch (Exception e) {
+ this.collector.reportError(e);
+ throw new RuntimeException("MQTT Connection failed.", e);
+ }
+
+ }
+
+ private void connectMqtt() throws Exception {
+ MQTT client = new MQTT();
--- End diff --
Code block from L119 to L148 is duplicated by MqttPublisher.connectMqtt(),
L65 to L93, except setting up client id.
We may want to extract duplicated codes to method of another new (util?)
class.
> MQTT Support
> ------------
>
> Key: STORM-1406
> URL: https://issues.apache.org/jira/browse/STORM-1406
> Project: Apache Storm
> Issue Type: Improvement
> Reporter: P. Taylor Goetz
> Assignee: P. Taylor Goetz
>
> MQTT is a lightweight publish/subscribe protocol frequently used in IoT
> applications.
> Further information can be found at http://mqtt.org
> Initial features include:
> * Full MQTT support (e.g. last will, QoS 0-2, retain, etc.)
> * Spout implementation(s) for subscribing to MQTT topics
> * A bolt implementation for publishing MQTT messages
> * A trident function implementation for publishing MQTT messages
> * Authentication and TLS/SSL support
> * User-defined "mappers" for converting MQTT messages to tuples (subscribers)
> * User-defined "mappers" for converting tuples to MQTT messages (publishers)
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)