ic4y commented on code in PR #2832: URL: https://github.com/apache/incubator-seatunnel/pull/2832#discussion_r1063938913
########## docs/en/connector-v2/sink/TDengine.md: ########## @@ -0,0 +1,69 @@ +# TDengine + +> TDengine sink connector + +## Description + +Used to write data to TDengine. You need to create stable before running seatunnel task + +## Key features + +- [x] [parallelism](../../concept/connector-v2-features.md) +- [ ] [support user-defined split](../../concept/connector-v2-features.md) Review Comment: Refer to docs/en/concept/connector-v2-features.md, sink only has two features exactly-once and cdc ########## docs/en/connector-v2/source/TDengine.md: ########## @@ -0,0 +1,82 @@ +# TDengine + +> TDengine source connector + +## Description + +Read external data source data through TDengine. + +## Key features + +- [x] [batch](../../concept/connector-v2-features.md) +- [ ] [stream](../../concept/connector-v2-features.md) +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [schema projection](../../concept/connector-v2-features.md) Review Comment: Refer to docs/en/concept/connector-v2-features.md, schema projection is currently called column projection ########## seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceReader.java: ########## @@ -0,0 +1,157 @@ +/* + * 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.seatunnel.connectors.seatunnel.tdengine.source; + +import org.apache.seatunnel.api.source.Boundedness; +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.tdengine.exception.TDengineConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.tdengine.exception.TDengineConnectorException; + +import com.google.common.collect.Sets; +import com.taosdata.jdbc.TSDBDriver; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; + +@Slf4j +public class TDengineSourceReader implements SourceReader<SeaTunnelRow, TDengineSourceSplit> { + + private static final long THREAD_WAIT_TIME = 500L; + + private final TDengineSourceConfig config; + + private final Set<TDengineSourceSplit> sourceSplits; + + private final Context context; + + private Connection conn; + + public TDengineSourceReader(TDengineSourceConfig config, SourceReader.Context readerContext) { + this.config = config; + this.sourceSplits = Sets.newHashSet(); + this.context = readerContext; + } + + @Override + public void pollNext(Collector<SeaTunnelRow> collector) throws InterruptedException { + if (sourceSplits.isEmpty()) { + Thread.sleep(THREAD_WAIT_TIME); + return; + } + sourceSplits.forEach(split -> { Review Comment: During the read split and changing state(sourceSplits), you need to obtain CheckpointLock, otherwise it cannot be guaranteed to be exactly-once. ########## seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceReader.java: ########## @@ -0,0 +1,157 @@ +/* + * 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.seatunnel.connectors.seatunnel.tdengine.source; + +import org.apache.seatunnel.api.source.Boundedness; +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.tdengine.exception.TDengineConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.tdengine.exception.TDengineConnectorException; + +import com.google.common.collect.Sets; +import com.taosdata.jdbc.TSDBDriver; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; + +@Slf4j +public class TDengineSourceReader implements SourceReader<SeaTunnelRow, TDengineSourceSplit> { + + private static final long THREAD_WAIT_TIME = 500L; + + private final TDengineSourceConfig config; + + private final Set<TDengineSourceSplit> sourceSplits; + + private final Context context; + + private Connection conn; + + public TDengineSourceReader(TDengineSourceConfig config, SourceReader.Context readerContext) { + this.config = config; + this.sourceSplits = Sets.newHashSet(); + this.context = readerContext; + } + + @Override + public void pollNext(Collector<SeaTunnelRow> collector) throws InterruptedException { + if (sourceSplits.isEmpty()) { + Thread.sleep(THREAD_WAIT_TIME); + return; + } + sourceSplits.forEach(split -> { Review Comment: like ``` synchronized (output.getCheckpointLock()) { sourceSplits.forEach(split -> { read(split, collector); } } ``` Or refer to JdbcSourceReader to use a smaller lock granularity ########## seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/config/TDengineSourceConfig.java: ########## @@ -0,0 +1,79 @@ +/* + * 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.seatunnel.connectors.seatunnel.tdengine.config; + +import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.DATABASE; +import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.LOWER_BOUND; +import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.PASSWORD; +import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.STABLE; +import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.TIMEZONE; +import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.UPPER_BOUND; +import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.URL; +import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.USERNAME; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import lombok.Data; + +import java.io.Serializable; +import java.util.List; + +@Data +public class TDengineSourceConfig implements Serializable { + + /** + * jdbc:TAOS-RS://localhost:6041/ + */ + private String url; + private String username; + private String password; + private String database; + private String stable; + //param of timezone in 'jdbc:TAOS-RS' just effect on taosadapter side, other than the JDBC client side + //so this param represent the server-side timezone setting up + private String timezone; + private String lowerBound; + private String upperBound; + private List<String> fields; + private List<String> tags; + + public static TDengineSourceConfig buildSourceConfig(Config pluginConfig) { + TDengineSourceConfig tdengineSourceConfig = new TDengineSourceConfig(); + tdengineSourceConfig.setUrl(pluginConfig.hasPath(URL) ? pluginConfig.getString(URL) : null); Review Comment: It is better to add config check。 ########## seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/sink/TDengineSinkWriter.java: ########## @@ -0,0 +1,116 @@ +/* + * 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.seatunnel.connectors.seatunnel.tdengine.sink; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; +import org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import com.taosdata.jdbc.TSDBDriver; +import lombok.SneakyThrows; +import org.apache.commons.lang3.ArrayUtils; +import org.apache.commons.lang3.StringUtils; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Objects; +import java.util.Properties; + +public class TDengineSinkWriter extends AbstractSinkWriter<SeaTunnelRow, Void> { + + private static final DateTimeFormatter FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS"); + private final SeaTunnelRowType seaTunnelRowType; + private final Connection conn; + private final TDengineSourceConfig config; + private int tagsNum; + + @SneakyThrows + public TDengineSinkWriter(Config pluginConfig, SeaTunnelRowType seaTunnelRowType) { + this.seaTunnelRowType = seaTunnelRowType; + this.tagsNum = pluginConfig.hasPath("tags") ? pluginConfig.getObject("tags").size() : 0; + + config = TDengineSourceConfig.buildSourceConfig(pluginConfig); + String jdbcUrl = StringUtils.join(config.getUrl(), config.getDatabase(), "?user=", config.getUsername(), "&password=", config.getPassword()); + Properties connProps = new Properties(); + connProps.setProperty(TSDBDriver.PROPERTY_KEY_BATCH_LOAD, "true"); + conn = DriverManager.getConnection(jdbcUrl, connProps); + } + + @SneakyThrows + @Override + @SuppressWarnings("checkstyle:RegexpSingleline") + public void write(SeaTunnelRow element) { + final ArrayList<Object> tags = Lists.newArrayList(); + for (int i = element.getArity() - tagsNum; i < element.getArity(); i++) { + tags.add(element.getField(i)); + } + final String tagValues = StringUtils.join(convertDataType(tags.toArray()), ","); + + final Object[] metrics = ArrayUtils.subarray(element.getFields(), 1, element.getArity() - tagsNum); + + try (Statement statement = conn.createStatement()) { + String sql = "INSERT INTO " + element.getField(0) + + " using " + config.getStable() + + " tags ( " + + tagValues + + " ) VALUES (" + + StringUtils.join(convertDataType(metrics), ",") + + ");"; + final int rowCount = statement.executeUpdate(sql); Review Comment: Whether to support `write(List<SeaTunnelRow>)` needs to be discussed. Currently ST only supports write (SeaTunnelRow), if you need better performance, you need to implement batch flush yourself. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
