Skip to content
This repository has been archived by the owner on Feb 16, 2024. It is now read-only.

[BAHIR-212] Add MQTT Connector for Flink #100

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 5 additions & 0 deletions distribution/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,11 @@
<artifactId>flink-connector-flume_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.bahir</groupId>
<artifactId>flink-connector-mqtt_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.bahir</groupId>
<artifactId>flink-connector-netty_${scala.binary.version}</artifactId>
Expand Down
15 changes: 15 additions & 0 deletions flink-connector-mqtt/README.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
# Flink Mqtt Connector

This connector provides a source and sink to [MQTT](https://mqtt.org/)™
To use this connector, add the following dependency to your project:

<dependency>
<groupId>org.apache.bahir</groupId>
<artifactId>flink-connector-mqtt_2.11</artifactId>
<version>1.1-SNAPSHOT</version>
</dependency>

Note that the streaming connectors are not part of the binary distribution of Flink. You need to link them into your job jar for cluster execution.
See how to link with them for cluster execution [here](https://ci.apache.org/projects/flink/flink-docs-release-1.2/dev/linking.html).

The source class is called `MqttSource`, and the sink is `MqttSink`.
69 changes: 69 additions & 0 deletions flink-connector-mqtt/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
<?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/maven-v4_0_0.xsd">

<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>org.apache.bahir</groupId>
<artifactId>bahir-flink-parent_2.11</artifactId>
<version>1.1-SNAPSHOT</version>
<relativePath>..</relativePath>
</parent>

<artifactId>flink-connector-mqtt_2.11</artifactId>
<name>flink-connector-mqtt</name>

<packaging>jar</packaging>

<properties>
<mqtt.paho.client.version>1.2.2</mqtt.paho.client.version>
</properties>

<dependencies>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>org.eclipse.paho</groupId>
<artifactId>org.eclipse.paho.client.mqttv3</artifactId>
<version>${mqtt.paho.client.version}</version>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-clients_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>

</dependencies>

</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/*
* 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.flink.streaming.connectors.mqtt;

import java.io.Serializable;
import java.util.Properties;

/**
* mqtt config, include required / optional property keys.
*/
public class MqttConfig implements Serializable {

// ----- Required property keys
protected static final String SERVER_URL = "server.url";
protected static final String USERNAME = "username";
protected static final String PASSWORD = "password";

// ------ Optional property keys
protected static final String CLIENT_ID = "client.id";
protected static final String CLEAN_SESSION = "clean.session";
protected static final String RETAINED = "retained";
protected static final String QOS = "qos";

protected static void checkProperty(Properties properties, String key) {
if (!properties.containsKey(key)) {
throw new IllegalArgumentException("Required property '" + key + "' not set.");
}
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,171 @@
/*
* 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.flink.streaming.connectors.mqtt;

import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
import org.eclipse.paho.client.mqttv3.MqttCallbackExtended;
import org.eclipse.paho.client.mqttv3.MqttClient;
import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
import org.eclipse.paho.client.mqttv3.MqttMessage;
import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.Properties;

import static java.util.Collections.singletonList;
import static org.apache.flink.shaded.guava18.com.google.common.collect.Iterables.getOnlyElement;
import static org.apache.flink.streaming.connectors.mqtt.MqttConfig.checkProperty;

/**
* Sink for writing messages to an Mqtt queue.
*
* @param <IN> type of input messages
*/
public class MqttSink<IN> extends RichSinkFunction<IN> implements CheckpointedFunction, MqttCallbackExtended {
private static final Logger LOG = LoggerFactory.getLogger(MqttSink.class);

// Mqtt publisher client id
private static final String MQTT_CLIENT_ID = MqttClient.generateClientId();

// Publish topic name
private final String topic;
// Convert input message to bytes
private final SerializationSchema<IN> serializationSchema;
// User-supplied mqtt properties
private final Properties properties;

private String serverUrl;
private String username;
private String password;
private String clientId;
private boolean cleanSession;
private boolean retained;
private int qos;

private transient ListState<String> mqttState;
private transient MqttClient client;

/**
* Creates {@link MqttSink} for Streaming
*
* @param topic Publish topic name
* @param serializationSchema Convert input message to bytes
* @param properties Mqtt properties
*/
public MqttSink(String topic,
SerializationSchema<IN> serializationSchema,
Properties properties) {
checkProperty(properties, MqttConfig.SERVER_URL);
checkProperty(properties, MqttConfig.USERNAME);
checkProperty(properties, MqttConfig.PASSWORD);

this.topic = topic;
this.serializationSchema = serializationSchema;
this.properties = properties;
}

@Override
public void initializeState(FunctionInitializationContext context) throws Exception {
this.mqttState = context.getOperatorStateStore().getListState(
new ListStateDescriptor<>("mqttClientId", String.class));
this.clientId = context.isRestored() ? getOnlyElement(mqttState.get()) : MQTT_CLIENT_ID;
LOG.info("Current mqtt publisher clientId is : {}.", clientId);
}

@Override
public void snapshotState(FunctionSnapshotContext context) throws Exception {
mqttState.update(singletonList(MQTT_CLIENT_ID));
}

@Override
public void open(Configuration parameters) throws Exception {
super.open(parameters);
this.serverUrl = properties.getProperty(MqttConfig.SERVER_URL);
this.username = properties.getProperty(MqttConfig.USERNAME);
this.password = properties.getProperty(MqttConfig.PASSWORD);
this.clientId = properties.getProperty(MqttConfig.CLIENT_ID, clientId);
this.cleanSession = Boolean.parseBoolean(properties.getProperty(
MqttConfig.CLEAN_SESSION, "true"));
this.retained = Boolean.parseBoolean(properties.getProperty(
MqttConfig.RETAINED, "false"));
this.qos = Integer.parseInt(properties.getProperty(
MqttConfig.QOS, "1"));

MqttConnectOptions options = new MqttConnectOptions();
options.setUserName(username);
options.setPassword(password.toCharArray());
options.setCleanSession(cleanSession);
options.setAutomaticReconnect(true);

client = new MqttClient(serverUrl, clientId, new MemoryPersistence());
client.setCallback(this);
// Connect to mqtt broker
client.connect(options);

LOG.info("Sink subtask {} initialize by serverUrl:{}, start publishing topic:{} with " +
"clientId:{} and qos:{}.", getRuntimeContext().getIndexOfThisSubtask(),
serverUrl, topic, clientId, qos);
}

@Override
public void invoke(IN value, Context context) throws Exception {
try {
byte[] message = serializationSchema.serialize(value);
client.publish(topic, message, qos, retained);
} catch (Exception e) {
LOG.error("Failed to publish message to mqtt broker, topic: {}, message: {}", topic, value);
throw new RuntimeException("Failed to publish message to mqtt broker", e);
}
}

@Override
public void close() throws Exception {
client.disconnect();
client.close();
}

@Override
public void connectComplete(boolean reconnect, String serUrl) {
LOG.info("Call connectComplete method, reconnect:{}, serUrl:{}.", reconnect, serUrl);
}

@Override
public void connectionLost(Throwable cause) {
LOG.error("Connection has losted between publisher org.apache.flink.streaming.connectors.mqtt client and broker.", cause);
cause.printStackTrace();
}

@Override
public void messageArrived(String topic, MqttMessage message) throws Exception {
throw new IllegalStateException("Method messageArrived is for subscriber, so should not go to here.");
}

@Override
public void deliveryComplete(IMqttDeliveryToken token) {
}

}
Loading