Skip to content

Commit

Permalink
[IOTDB-497] Apache Flink Connector Support (#817)
Browse files Browse the repository at this point in the history
  • Loading branch information
vesense committed Mar 7, 2020
1 parent 27fb440 commit f7c850f
Show file tree
Hide file tree
Showing 16 changed files with 1,091 additions and 1 deletion.
30 changes: 30 additions & 0 deletions example/flink/README.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
<!--
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.
-->
# IoTDB-Flink-Connector Example

## Function
```
The example is to show how to send data to a IoTDB server from a Flink job.
```

## Usage

* Run `org.apache.iotdb.flink.FlinkIoTDBSink.java` to launch the local iotDB server and run the flink job on local mini cluster.
45 changes: 45 additions & 0 deletions example/flink/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
<?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.iotdb</groupId>
<artifactId>iotdb-examples</artifactId>
<version>0.10.0-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<artifactId>flink-example</artifactId>
<name>IoTDB-Flink Examples</name>
<packaging>jar</packaging>
<dependencies>
<dependency>
<groupId>org.apache.iotdb</groupId>
<artifactId>flink-iotdb-connector</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.iotdb</groupId>
<artifactId>iotdb-server</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
/*
* 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.iotdb.flink;

import com.google.common.collect.Lists;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.iotdb.db.service.IoTDB;

import java.util.HashMap;
import java.util.Map;
import java.util.Random;

public class FlinkIoTDBSink {
public static void main(String[] args) throws Exception {
// launch the local iotDB server at default port: 6667
IoTDB.main(args);

Thread.sleep(3000);

// run the flink job on local mini cluster
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();

IoTDBOptions options = new IoTDBOptions();
options.setHost("127.0.0.1");
options.setPort(6667);
options.setUser("root");
options.setPassword("root");
options.setStorageGroup("root.sg");
options.setTimeseriesOptionList(Lists.newArrayList(new IoTDBOptions.TimeseriesOption("root.sg.d1.s1")));

IoTSerializationSchema serializationSchema = new DefaultIoTSerializationSchema();
IoTDBSink ioTDBSink = new IoTDBSink(options, serializationSchema)
// enable batching
.withBatchSize(10);

env.addSource(new SensorSource())
.name("sensor-source")
.setParallelism(1)
.addSink(ioTDBSink)
.name("iotdb-sink")
.setParallelism(1);

env.execute("iotdb-flink-example");
}

private static class SensorSource implements SourceFunction<Map<String,String>> {
boolean running = true;

@Override
public void run(SourceContext context) throws Exception {
Random random = new Random();
while (running) {
Map<String,String> tuple = new HashMap();
tuple.put("device", "root.sg.d1");
tuple.put("timestamp", String.valueOf(System.currentTimeMillis()));
tuple.put("measurements", "s1");
tuple.put("values", String.valueOf(random.nextDouble()));

context.collect(tuple);
Thread.sleep(1000);
}
}

@Override
public void cancel() {
running = false;
}
}
}
1 change: 1 addition & 0 deletions example/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
<module>tsfile</module>
<module>jdbc</module>
<module>hadoop</module>
<module>flink</module>
</modules>
<build>
<pluginManagement>
Expand Down
58 changes: 58 additions & 0 deletions flink-iotdb-connector/README.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
<!--
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.
-->
# IoTDB-Flink-Connector

IoTDB integration for [Apache Flink](https://flink.apache.org/). This module includes the iotdb sink that allows a flink job to write events into timeseries.

## IoTDBSink
To use the `IoTDBSink`, you need construct an instance of it by specifying `IoTDBOptions` and `IoTSerializationSchema` instances.
The `IoTDBSink` send only one event after another by default, but you can change to batch by invoking `withBatchSize(int)`.

## Examples
The following is an example which receiving events from sensor source and then sending events to iotdb.

```java
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();

IoTDBOptions options = new IoTDBOptions();
options.setHost("127.0.0.1");
options.setPort(6667);
options.setUser("root");
options.setPassword("root");
options.setStorageGroup("root.sg");
options.setTimeseries(Lists.newArrayList("root.sg.d1.s1"));

IoTSerializationSchema serializationSchema = new DefaultIoTSerializationSchema();
IoTDBSink ioTDBSink = new IoTDBSink(options, serializationSchema)
// enable batching
.withBatchSize(10)
;

env.addSource(new SensorSource())
.name("sensor-source")
.setParallelism(1)
.addSink(ioTDBSink)
.name("iotdb-sink")
.setParallelism(1)
;

env.execute("iotdb-flink-example");
```
55 changes: 55 additions & 0 deletions flink-iotdb-connector/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
<?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.iotdb</groupId>
<artifactId>iotdb-parent</artifactId>
<version>0.10.0-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<artifactId>flink-iotdb-connector</artifactId>
<packaging>jar</packaging>
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<compile.version>1.8</compile.version>
<flink.version>1.10.0</flink.version>
<scala.binary.version>2.11</scala.binary.version>
</properties>
<dependencies>
<dependency>
<groupId>org.apache.iotdb</groupId>
<artifactId>iotdb-session</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
<dependency>
<groupId>org.powermock</groupId>
<artifactId>powermock-module-junit4</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
/*
* 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.iotdb.flink;

import java.util.Arrays;
import java.util.List;
import java.util.Map;

/**
* @inheritDoc
* The default implementation of IoTSerializationSchema. Gets info from a map struct.
*/
public class DefaultIoTSerializationSchema implements IoTSerializationSchema<Map<String,String>> {
private String fieldDevice = "device";
private String fieldTimestamp = "timestamp";
private String fieldMeasurements = "measurements";
private String fieldValues = "values";
private String separator = ",";

@Override
public Event serialize(Map<String,String> tuple) {
if (tuple == null) {
return null;
}

String device = tuple.get(fieldDevice);

String ts = tuple.get(fieldTimestamp);
Long timestamp = ts == null ? System.currentTimeMillis() : Long.parseLong(ts);

List<String> measurements = null;
if (tuple.get(fieldMeasurements) != null) {
measurements = Arrays.asList(tuple.get(fieldMeasurements).split(separator));
}

List<String> values = null;
if (tuple.get(fieldValues) != null) {
values = Arrays.asList(tuple.get(fieldValues).split(separator));
}

return new Event(device, timestamp, measurements, values);
}

public String getFieldDevice() {
return fieldDevice;
}

public void setFieldDevice(String fieldDevice) {
this.fieldDevice = fieldDevice;
}

public String getFieldTimestamp() {
return fieldTimestamp;
}

public void setFieldTimestamp(String fieldTimestamp) {
this.fieldTimestamp = fieldTimestamp;
}

public String getFieldMeasurements() {
return fieldMeasurements;
}

public void setFieldMeasurements(String fieldMeasurements) {
this.fieldMeasurements = fieldMeasurements;
}

public String getFieldValues() {
return fieldValues;
}

public void setFieldValues(String fieldValues) {
this.fieldValues = fieldValues;
}

public String getSeparator() {
return separator;
}

public void setSeparator(String separator) {
this.separator = separator;
}
}
Loading

0 comments on commit f7c850f

Please sign in to comment.