Skip to content
Closed
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
68 changes: 68 additions & 0 deletions iotdb-connector/flink-sql-iotdb-connector/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
<?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>1.2.0-SNAPSHOT</version>
<relativePath>../../pom.xml</relativePath>
</parent>
<artifactId>flink-sql-iotdb-connector</artifactId>
<name>IoTDB: Connector: Apache Flink SQL Connector</name>
<version>1.2.0-SNAPSHOT</version>
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<flink.version>1.17.0</flink.version>
</properties>
<dependencies>
<!-- required by implement of flink sql connector -->
<dependency>
<groupId>org.apache.iotdb</groupId>
<artifactId>iotdb-session</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-java-bridge</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-common</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>org.java-websocket</groupId>
<artifactId>Java-WebSocket</artifactId>
<version>${websocket.version}</version>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* 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.sql.client;

import org.apache.iotdb.flink.sql.function.IoTDBCDCSourceFunction;
import org.apache.iotdb.flink.sql.wrapper.TabletWrapper;
import org.apache.iotdb.tsfile.write.record.Tablet;

import org.java_websocket.client.WebSocketClient;
import org.java_websocket.handshake.ServerHandshake;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.net.URI;
import java.nio.ByteBuffer;

public class IoTDBWebSocketClient extends WebSocketClient {
private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBWebSocketClient.class);
private final IoTDBCDCSourceFunction function;

public IoTDBWebSocketClient(URI uri, IoTDBCDCSourceFunction function) {
super(uri);
this.function = function;
}

@Override
public void onOpen(ServerHandshake serverHandshake) {
String log =
String.format("The connection with %s:%d has been created.", uri.getHost(), uri.getPort());
LOGGER.info(log);
}

@Override
public void onMessage(String s) {
// Do nothing
}

@Override
public void onMessage(ByteBuffer bytes) {
super.onMessage(bytes);
long commitId = bytes.getLong();
Tablet tablet = Tablet.deserialize(bytes);
function.addTabletWrapper(new TabletWrapper(commitId, this, tablet));
}

@Override
public void onClose(int i, String s, boolean b) {
LOGGER.info("The connection to {}:{} has been closed.", uri.getHost(), uri.getPort());
}

@Override
public void onError(Exception e) {
String log =
String.format(
"An error occurred when connecting to %s:%s: %s.",
uri.getHost(), uri.getPort(), e.getMessage());
LOGGER.error(log);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* 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.sql.common;

import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;

public class Options {
public static final ConfigOption<String> NODE_URLS =
ConfigOptions.key("nodeUrls").stringType().defaultValue("127.0.0.1:6667");
public static final ConfigOption<String> USER =
ConfigOptions.key("user").stringType().defaultValue("root");
public static final ConfigOption<String> PASSWORD =
ConfigOptions.key("password").stringType().defaultValue("root");
public static final ConfigOption<String> DEVICE =
ConfigOptions.key("device").stringType().noDefaultValue();
public static final ConfigOption<Boolean> ALIGNED =
ConfigOptions.key("aligned").booleanType().defaultValue(false);
public static final ConfigOption<Mode> MODE =
ConfigOptions.key("mode").enumType(Mode.class).defaultValue(Mode.BOUNDED);
public static final ConfigOption<Integer> CDC_PORT =
ConfigOptions.key("cdc.port").intType().defaultValue(8080);

public static final ConfigOption<String> CDC_TASK_NAME =
ConfigOptions.key("cdc.task.name").stringType().noDefaultValue();
public static final ConfigOption<Integer> LOOKUP_CACHE_MAX_ROWS =
ConfigOptions.key("lookup.cache.max-rows").intType().defaultValue(-1);
public static final ConfigOption<Integer> LOOKUP_CACHE_TTL_SEC =
ConfigOptions.key("lookup.cache.ttl-sec").intType().defaultValue(-1);
public static final ConfigOption<Long> SCAN_BOUNDED_LOWER_BOUND =
ConfigOptions.key("scan.bounded.lower-bound").longType().defaultValue(-1L);
public static final ConfigOption<Long> SCAN_BOUNDED_UPPER_BOUND =
ConfigOptions.key("scan.bounded.upper-bound").longType().defaultValue(-1L);

public enum Mode {
CDC,
BOUNDED;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,144 @@
/*
* 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.sql.common;

import org.apache.iotdb.flink.sql.exception.UnsupportedDataTypeException;
import org.apache.iotdb.tsfile.exception.NullFieldException;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.Field;
import org.apache.iotdb.tsfile.read.common.RowRecord;
import org.apache.iotdb.tsfile.utils.Binary;

import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.StringData;
import org.apache.flink.table.types.DataType;

import java.io.IOException;
import java.lang.reflect.Array;
import java.net.Socket;
import java.net.URI;
import java.util.ArrayList;
import java.util.List;
import java.util.regex.Pattern;

public class Utils {
private Utils() {}

public static Object getValue(Field value, String dataType) {
try {
if ("INT32".equals(dataType)) {
return value.getIntV();
} else if ("INT64".equals(dataType)) {
return value.getLongV();
} else if ("FLOAT".equals(dataType)) {
return value.getFloatV();
} else if ("DOUBLE".equals(dataType)) {
return value.getDoubleV();
} else if ("BOOLEAN".equals(dataType)) {
return value.getBoolV();
} else if ("TEXT".equals(dataType)) {
return StringData.fromString(value.getStringValue());
} else {
String exception = String.format("IoTDB doesn't support the data type: %s", dataType);
throw new UnsupportedDataTypeException(exception);
}
} catch (NullFieldException e) {
return null;
}
}

public static Object getValue(Field value, DataType dataType) {
if (dataType.equals(DataTypes.INT())) {
return value.getIntV();
} else if (dataType.equals(DataTypes.BIGINT())) {
return value.getLongV();
} else if (dataType.equals(DataTypes.FLOAT())) {
return value.getFloatV();
} else if (dataType.equals(DataTypes.DOUBLE())) {
return value.getDoubleV();
} else if (dataType.equals(DataTypes.BOOLEAN())) {
return value.getBoolV();
} else if (dataType.equals(DataTypes.STRING())) {
return StringData.fromString(value.getStringValue());
} else {
throw new UnsupportedDataTypeException("IoTDB doesn't support the data type: " + dataType);
}
}

public static Object getValue(RowData value, DataType dataType, int index) {
try {
if (dataType.equals(DataTypes.INT())) {
return value.getInt(index);
} else if (dataType.equals(DataTypes.BIGINT())) {
return value.getLong(index);
} else if (dataType.equals(DataTypes.FLOAT())) {
return value.getFloat(index);
} else if (dataType.equals(DataTypes.DOUBLE())) {
return value.getDouble(index);
} else if (dataType.equals(DataTypes.BOOLEAN())) {
return value.getBoolean(index);
} else if (dataType.equals(DataTypes.STRING())) {
return value.getString(index).toString();
} else {
throw new UnsupportedDataTypeException("IoTDB don't support the data type: " + dataType);
}
} catch (NullPointerException e) {
return null;
}
}

public static boolean isNumeric(String s) {
Pattern pattern = Pattern.compile("\\d*");
return pattern.matcher(s).matches();
}

public static RowData convert(RowRecord rowRecord, List<String> columnTypes) {
ArrayList<Object> values = new ArrayList<>();
values.add(rowRecord.getTimestamp());
List<Field> fields = rowRecord.getFields();
for (int i = 0; i < fields.size(); i++) {
values.add(getValue(fields.get(i), columnTypes.get(i + 1)));
}
return GenericRowData.of(values.toArray());
}

public static List<Object> object2List(Object obj, TSDataType dataType) {
ArrayList<Object> objects = new ArrayList<>();
int length = Array.getLength(obj);
for (int i = 0; i < length; i++) {
if (dataType == TSDataType.TEXT) {
objects.add(StringData.fromString(((Binary) Array.get(obj, i)).getStringValue()));
} else {
objects.add(Array.get(obj, i));
}
}
return objects;
}

public static boolean isURIAvailable(URI uri) {
try {
new Socket(uri.getHost(), uri.getPort()).close();
return true;
} catch (IOException e) {
return false;
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* 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.sql.exception;

public class IllegalIoTDBPathException extends RuntimeException {
public IllegalIoTDBPathException(String s) {
super(s);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* 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.sql.exception;

public class IllegalOptionException extends RuntimeException {
public IllegalOptionException(String s) {
super(s);
}
}
Loading