Skip to content
Merged
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
12 changes: 6 additions & 6 deletions connectors/flink-iotdb-connector/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@
<parent>
<groupId>org.apache.iotdb</groupId>
<artifactId>connectors</artifactId>
<version>1.3.2</version>
<version>2.0.2-SNAPSHOT</version>
</parent>
<artifactId>flink-iotdb-connector</artifactId>
<name>IoTDB: Connector: Apache Flink</name>
Expand All @@ -40,14 +40,14 @@
<version>${iotdb.version}</version>
</dependency>
<dependency>
<groupId>org.apache.iotdb</groupId>
<artifactId>common-api</artifactId>
<version>${iotdb.version}</version>
<groupId>org.apache.tsfile</groupId>
<artifactId>common</artifactId>
<version>${tsfile.version}</version>
</dependency>
<dependency>
<groupId>org.apache.iotdb</groupId>
<groupId>org.apache.tsfile</groupId>
<artifactId>tsfile</artifactId>
<version>${iotdb.version}</version>
<version>${tsfile.version}</version>
</dependency>
<dependency>
<groupId>org.apache.iotdb</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@

package org.apache.iotdb.flink;

import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.tsfile.enums.TSDataType;

import java.util.ArrayList;
import java.util.Arrays;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@

package org.apache.iotdb.flink;

import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.tsfile.enums.TSDataType;

import java.util.List;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,12 @@

import org.apache.iotdb.flink.options.IoTDBSinkOptions;
import org.apache.iotdb.session.pool.SessionPool;
import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;

import com.google.common.base.Preconditions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
import org.apache.tsfile.common.constant.TsFileConstant;
import org.apache.tsfile.enums.TSDataType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,10 +23,10 @@
import org.apache.iotdb.rpc.IoTDBConnectionException;
import org.apache.iotdb.rpc.StatementExecutionException;
import org.apache.iotdb.session.Session;
import org.apache.iotdb.tsfile.read.common.RowRecord;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
import org.apache.tsfile.read.common.RowRecord;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,9 @@

package org.apache.iotdb.flink.options;

import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.tsfile.enums.TSDataType;
import org.apache.tsfile.file.metadata.enums.CompressionType;
import org.apache.tsfile.file.metadata.enums.TSEncoding;

import java.io.Serializable;
import java.util.List;
Expand Down
12 changes: 6 additions & 6 deletions connectors/flink-sql-iotdb-connector/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.iotdb</groupId>
<artifactId>connectors</artifactId>
<version>1.3.2</version>
<version>2.0.2-SNAPSHOT</version>
</parent>
<artifactId>flink-sql-iotdb-connector</artifactId>
<name>IoTDB: Connector: Apache Flink SQL</name>
Expand All @@ -51,14 +51,14 @@
<version>${iotdb.version}</version>
</dependency>
<dependency>
<groupId>org.apache.iotdb</groupId>
<artifactId>common-api</artifactId>
<version>${iotdb.version}</version>
<groupId>org.apache.tsfile</groupId>
<artifactId>common</artifactId>
<version>${tsfile.version}</version>
</dependency>
<dependency>
<groupId>org.apache.iotdb</groupId>
<groupId>org.apache.tsfile</groupId>
<artifactId>tsfile</artifactId>
<version>${iotdb.version}</version>
<version>${tsfile.version}</version>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,8 @@

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.apache.tsfile.write.record.Tablet;
import org.java_websocket.client.WebSocketClient;
import org.java_websocket.handshake.ServerHandshake;
import org.slf4j.Logger;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,19 +19,19 @@
package org.apache.iotdb.flink.sql.common;

import org.apache.iotdb.flink.sql.exception.UnsupportedDataTypeException;
import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
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.api.java.tuple.Tuple2;
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 org.apache.tsfile.common.conf.TSFileConfig;
import org.apache.tsfile.enums.TSDataType;
import org.apache.tsfile.exception.NullFieldException;
import org.apache.tsfile.read.common.Field;
import org.apache.tsfile.read.common.RowRecord;
import org.apache.tsfile.utils.Binary;

import javax.annotation.Nullable;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,6 @@
import org.apache.iotdb.rpc.IoTDBConnectionException;
import org.apache.iotdb.rpc.StatementExecutionException;
import org.apache.iotdb.session.Session;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.RowRecord;

import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
import org.apache.flink.api.common.io.RichInputFormat;
Expand All @@ -40,6 +38,8 @@
import org.apache.flink.core.io.InputSplitAssigner;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.DataType;
import org.apache.tsfile.enums.TSDataType;
import org.apache.tsfile.read.common.RowRecord;

import java.util.Arrays;
import java.util.List;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,12 +28,6 @@
import org.apache.iotdb.flink.sql.wrapper.TabletWrapper;
import org.apache.iotdb.isession.SessionDataSet;
import org.apache.iotdb.session.Session;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.RowRecord;
import org.apache.iotdb.tsfile.utils.BitMap;
import org.apache.iotdb.tsfile.utils.Pair;
import org.apache.iotdb.tsfile.write.record.Tablet;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;

import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
Expand All @@ -42,6 +36,12 @@
import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.DataType;
import org.apache.tsfile.enums.TSDataType;
import org.apache.tsfile.read.common.RowRecord;
import org.apache.tsfile.utils.BitMap;
import org.apache.tsfile.utils.Pair;
import org.apache.tsfile.write.record.Tablet;
import org.apache.tsfile.write.schema.IMeasurementSchema;
import org.java_websocket.client.WebSocketClient;
import org.java_websocket.enums.ReadyState;
import org.slf4j.Logger;
Expand Down Expand Up @@ -232,11 +232,11 @@ private IoTDBWebSocketClient initAndGet(URI uri) throws InterruptedException {
}

public void collectTablet(Tablet tablet, SourceContext<RowData> ctx) {
List<MeasurementSchema> schemas = tablet.getSchemas();
int rowSize = tablet.rowSize;
List<IMeasurementSchema> schemas = tablet.getSchemas();
int rowSize = tablet.getRowSize();
HashMap<String, Pair<BitMap, List<Object>>> values = new HashMap<>();
for (MeasurementSchema schema : schemas) {
String timeseries = String.format("%s.%s", tablet.deviceId, schema.getMeasurementId());
for (IMeasurementSchema schema : schemas) {
String timeseries = String.format("%s.%s", tablet.getDeviceId(), schema.getMeasurementName());
TSDataType iotdbType = schema.getType();
int index = timeseriesList.indexOf(timeseries);
if (index == -1) {
Expand All @@ -251,12 +251,12 @@ public void collectTablet(Tablet tablet, SourceContext<RowData> ctx) {
values.put(
timeseries,
new Pair<>(
tablet.bitMaps[schemas.indexOf(schema)],
Utils.object2List(tablet.values[schemas.indexOf(schema)], iotdbType)));
tablet.getBitMaps()[schemas.indexOf(schema)],
Utils.object2List(tablet.getValues()[schemas.indexOf(schema)], iotdbType)));
}
for (int i = 0; i < rowSize; i++) {
ArrayList<Object> row = new ArrayList<>();
row.add(tablet.timestamps[i]);
row.add(tablet.getTimestamps()[i]);
for (String timeseries : timeseriesList) {
if (values.containsKey(timeseries)
&& (values.get(timeseries).getLeft() == null
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,9 +26,6 @@
import org.apache.iotdb.rpc.IoTDBConnectionException;
import org.apache.iotdb.rpc.StatementExecutionException;
import org.apache.iotdb.session.Session;
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 com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
Expand All @@ -39,6 +36,9 @@
import org.apache.flink.table.functions.FunctionContext;
import org.apache.flink.table.functions.TableFunction;
import org.apache.flink.table.types.DataType;
import org.apache.tsfile.enums.TSDataType;
import org.apache.tsfile.read.common.Field;
import org.apache.tsfile.read.common.RowRecord;

import java.util.ArrayList;
import java.util.Arrays;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import org.apache.iotdb.flink.sql.exception.IllegalIoTDBPathException;
import org.apache.iotdb.flink.sql.wrapper.SchemaWrapper;
import org.apache.iotdb.session.Session;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;

import org.apache.commons.lang3.StringUtils;
import org.apache.flink.api.java.tuple.Tuple2;
Expand All @@ -35,6 +34,7 @@
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.DataType;
import org.apache.flink.types.RowKind;
import org.apache.tsfile.enums.TSDataType;

import java.util.ArrayList;
import java.util.Arrays;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,8 @@
package org.apache.iotdb.flink.sql.wrapper;

import org.apache.iotdb.flink.sql.client.IoTDBWebSocketClient;
import org.apache.iotdb.tsfile.write.record.Tablet;

import org.apache.tsfile.write.record.Tablet;

public class TabletWrapper {
private final long commitId;
Expand Down
14 changes: 7 additions & 7 deletions connectors/flink-tsfile-connector/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -24,20 +24,20 @@
<parent>
<groupId>org.apache.iotdb</groupId>
<artifactId>connectors</artifactId>
<version>1.3.2</version>
<version>2.0.2-SNAPSHOT</version>
</parent>
<artifactId>flink-tsfile-connector</artifactId>
<name>IoTDB: Connector: Apache Flink TSFile</name>
<dependencies>
<dependency>
<groupId>org.apache.iotdb</groupId>
<artifactId>common-api</artifactId>
<version>${iotdb.version}</version>
<groupId>org.apache.tsfile</groupId>
<artifactId>common</artifactId>
<version>${tsfile.version}</version>
</dependency>
<dependency>
<groupId>org.apache.iotdb</groupId>
<groupId>org.apache.tsfile</groupId>
<artifactId>tsfile</artifactId>
<version>${iotdb.version}</version>
<version>${tsfile.version}</version>
<exclusions>
<!--
We need the flink-shaded-zookeeper-3 dependency for some shaded classes.
Expand All @@ -54,7 +54,7 @@
<dependency>
<groupId>org.apache.iotdb</groupId>
<artifactId>hadoop-tsfile</artifactId>
<version>${iotdb.version}</version>
<version>2.0.2-SNAPSHOT</version>
<exclusions>
<!--
We need the flink-shaded-zookeeper-3 dependency for some shaded classes.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,8 @@

package org.apache.iotdb.flink.tsfile;

import org.apache.iotdb.tsfile.read.common.RowRecord;

import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
import org.apache.tsfile.read.common.RowRecord;

import java.io.Serializable;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,15 +19,14 @@

package org.apache.iotdb.flink.tsfile;

import org.apache.iotdb.tsfile.common.constant.QueryConstant;
import org.apache.iotdb.tsfile.read.common.Field;
import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.read.common.RowRecord;

import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.types.Row;
import org.apache.tsfile.common.constant.QueryConstant;
import org.apache.tsfile.read.common.Field;
import org.apache.tsfile.read.common.Path;
import org.apache.tsfile.read.common.RowRecord;

import java.util.List;
import java.util.stream.Collectors;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,23 +19,22 @@

package org.apache.iotdb.flink.tsfile;

import org.apache.iotdb.tsfile.common.constant.QueryConstant;
import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
import org.apache.iotdb.tsfile.utils.BytesUtils;
import org.apache.iotdb.tsfile.write.record.TSRecord;
import org.apache.iotdb.tsfile.write.record.datapoint.BooleanDataPoint;
import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
import org.apache.iotdb.tsfile.write.record.datapoint.DoubleDataPoint;
import org.apache.iotdb.tsfile.write.record.datapoint.FloatDataPoint;
import org.apache.iotdb.tsfile.write.record.datapoint.IntDataPoint;
import org.apache.iotdb.tsfile.write.record.datapoint.LongDataPoint;
import org.apache.iotdb.tsfile.write.record.datapoint.StringDataPoint;
import org.apache.iotdb.tsfile.write.schema.Schema;

import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.types.Row;
import org.apache.flink.util.Collector;
import org.apache.tsfile.common.constant.QueryConstant;
import org.apache.tsfile.common.constant.TsFileConstant;
import org.apache.tsfile.utils.BytesUtils;
import org.apache.tsfile.write.UnSupportedDataTypeException;
import org.apache.tsfile.write.record.TSRecord;
import org.apache.tsfile.write.record.datapoint.BooleanDataPoint;
import org.apache.tsfile.write.record.datapoint.DataPoint;
import org.apache.tsfile.write.record.datapoint.DoubleDataPoint;
import org.apache.tsfile.write.record.datapoint.FloatDataPoint;
import org.apache.tsfile.write.record.datapoint.IntDataPoint;
import org.apache.tsfile.write.record.datapoint.LongDataPoint;
import org.apache.tsfile.write.record.datapoint.StringDataPoint;
import org.apache.tsfile.write.schema.Schema;

import java.util.ArrayList;
import java.util.List;
Expand Down Expand Up @@ -79,7 +78,7 @@ public void open(Schema schema) {
.collect(Collectors.toList())
.indexOf(deviceId);
if (tsRecordIndex < 0) {
outputTemplateList.add(new TSRecord(0, deviceId));
outputTemplateList.add(new TSRecord(deviceId, 0));
tsRecordIndex = outputTemplateList.size() - 1;
}
tsRecordIndexMapping[i] = tsRecordIndex;
Expand All @@ -106,7 +105,7 @@ public void open(Schema schema) {

reuse = new TSRecord[outputTemplate.length];
for (int i = 0; i < outputTemplate.length; i++) {
reuse[i] = new TSRecord(0, outputTemplate[i].deviceId);
reuse[i] = new TSRecord(outputTemplate[i].deviceId, 0);
}
}

Expand Down
Loading
Loading