Skip to content

Commit a18fca8

Browse files
authored
[Fix][Connector-tdengine] Fix sql exception and concurrentmodifyexception when connect to taos and read data
1 parent 064fcad commit a18fca8

File tree

6 files changed

+279
-145
lines changed

6 files changed

+279
-145
lines changed

seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/config/TDengineSourceConfig.java

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,6 @@
3030
import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.STABLE;
3131
import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.TIMEZONE;
3232
import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.UPPER_BOUND;
33-
import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.URL;
3433
import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.USERNAME;
3534

3635
@Data
@@ -54,7 +53,10 @@ public class TDengineSourceConfig implements Serializable {
5453

5554
public static TDengineSourceConfig buildSourceConfig(Config pluginConfig) {
5655
TDengineSourceConfig tdengineSourceConfig = new TDengineSourceConfig();
57-
tdengineSourceConfig.setUrl(pluginConfig.hasPath(URL) ? pluginConfig.getString(URL) : null);
56+
tdengineSourceConfig.setUrl(
57+
pluginConfig.hasPath(ConfigNames.URL)
58+
? pluginConfig.getString(ConfigNames.URL)
59+
: null);
5860
tdengineSourceConfig.setDatabase(
5961
pluginConfig.hasPath(DATABASE) ? pluginConfig.getString(DATABASE) : null);
6062
tdengineSourceConfig.setStable(
@@ -69,6 +71,7 @@ public static TDengineSourceConfig buildSourceConfig(Config pluginConfig) {
6971
pluginConfig.hasPath(LOWER_BOUND) ? pluginConfig.getString(LOWER_BOUND) : null);
7072
tdengineSourceConfig.setTimezone(
7173
pluginConfig.hasPath(TIMEZONE) ? pluginConfig.getString(TIMEZONE) : "UTC");
74+
7275
return tdengineSourceConfig;
7376
}
7477

seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSource.java

Lines changed: 26 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,7 @@
4040
import org.apache.commons.lang3.ArrayUtils;
4141

4242
import com.google.auto.service.AutoService;
43+
import com.taosdata.jdbc.TSDBDriver;
4344
import lombok.SneakyThrows;
4445

4546
import java.sql.Connection;
@@ -49,6 +50,7 @@
4950
import java.sql.Statement;
5051
import java.util.ArrayList;
5152
import java.util.List;
53+
import java.util.Properties;
5254

5355
import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.DATABASE;
5456
import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.PASSWORD;
@@ -127,42 +129,36 @@ private StableMetadata getStableMetadata(TDengineSourceConfig config) throws SQL
127129
List<String> fieldNames = new ArrayList<>();
128130
List<SeaTunnelDataType<?>> fieldTypes = new ArrayList<>();
129131

130-
String jdbcUrl =
131-
String.join(
132-
"",
133-
config.getUrl(),
134-
config.getDatabase(),
135-
"?user=",
136-
config.getUsername(),
137-
"&password=",
138-
config.getPassword());
132+
String jdbcUrl = String.join("", config.getUrl(), config.getDatabase());
133+
139134
// check td driver whether exist and if not, try to register
140135
checkDriverExist(jdbcUrl);
141-
try (Connection conn = DriverManager.getConnection(jdbcUrl)) {
142-
try (Statement statement = conn.createStatement()) {
136+
137+
Properties properties = new Properties();
138+
properties.put(TSDBDriver.PROPERTY_KEY_USER, config.getUsername());
139+
properties.put(TSDBDriver.PROPERTY_KEY_PASSWORD, config.getPassword());
140+
String metaSQL =
141+
String.format(
142+
"select table_name from information_schema.ins_tables where db_name = '%s' and stable_name='%s'",
143+
config.getDatabase(), config.getStable());
144+
try (Connection conn = DriverManager.getConnection(jdbcUrl, properties);
145+
Statement statement = conn.createStatement();
143146
ResultSet metaResultSet =
144147
statement.executeQuery(
145-
"desc " + config.getDatabase() + "." + config.getStable());
146-
while (metaResultSet.next()) {
147-
if (timestampFieldName == null) {
148-
timestampFieldName = metaResultSet.getString(1);
149-
}
150-
fieldNames.add(metaResultSet.getString(1));
151-
fieldTypes.add(TDengineTypeMapper.mapping(metaResultSet.getString(2)));
148+
String.format(
149+
"desc %s.%s", config.getDatabase(), config.getStable()));
150+
ResultSet subTableNameResultSet = statement.executeQuery(metaSQL)) {
151+
while (metaResultSet.next()) {
152+
if (timestampFieldName == null) {
153+
timestampFieldName = metaResultSet.getString(1);
152154
}
155+
fieldNames.add(metaResultSet.getString(1));
156+
fieldTypes.add(TDengineTypeMapper.mapping(metaResultSet.getString(2)));
153157
}
154-
try (Statement statement = conn.createStatement()) {
155-
String metaSQL =
156-
"select table_name from information_schema.ins_tables where db_name = '"
157-
+ config.getDatabase()
158-
+ "' and stable_name='"
159-
+ config.getStable()
160-
+ "';";
161-
ResultSet subTableNameResultSet = statement.executeQuery(metaSQL);
162-
while (subTableNameResultSet.next()) {
163-
String subTableName = subTableNameResultSet.getString(1);
164-
subTableNames.add(subTableName);
165-
}
158+
159+
while (subTableNameResultSet.next()) {
160+
String subTableName = subTableNameResultSet.getString(1);
161+
subTableNames.add(subTableName);
166162
}
167163
}
168164

seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceReader.java

Lines changed: 43 additions & 52 deletions
Original file line numberDiff line numberDiff line change
@@ -17,17 +17,13 @@
1717

1818
package org.apache.seatunnel.connectors.seatunnel.tdengine.source;
1919

20-
import org.apache.seatunnel.api.source.Boundedness;
2120
import org.apache.seatunnel.api.source.Collector;
2221
import org.apache.seatunnel.api.source.SourceReader;
2322
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
2423
import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated;
2524
import org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig;
2625
import org.apache.seatunnel.connectors.seatunnel.tdengine.exception.TDengineConnectorException;
2726

28-
import org.apache.commons.lang3.StringUtils;
29-
30-
import com.google.common.collect.Sets;
3127
import com.taosdata.jdbc.TSDBDriver;
3228
import lombok.extern.slf4j.Slf4j;
3329

@@ -39,84 +35,76 @@
3935
import java.sql.Statement;
4036
import java.sql.Timestamp;
4137
import java.util.ArrayList;
38+
import java.util.Deque;
4239
import java.util.List;
4340
import java.util.Objects;
4441
import java.util.Properties;
45-
import java.util.Set;
42+
import java.util.concurrent.ConcurrentLinkedDeque;
4643

4744
import static org.apache.seatunnel.connectors.seatunnel.tdengine.utils.TDengineUtil.checkDriverExist;
4845

4946
@Slf4j
5047
public class TDengineSourceReader implements SourceReader<SeaTunnelRow, TDengineSourceSplit> {
51-
52-
private static final long THREAD_WAIT_TIME = 500L;
53-
5448
private final TDengineSourceConfig config;
5549

56-
private final Set<TDengineSourceSplit> sourceSplits;
50+
private final Deque<TDengineSourceSplit> sourceSplits;
5751

5852
private final Context context;
5953

6054
private Connection conn;
6155

56+
private volatile boolean noMoreSplit;
57+
6258
public TDengineSourceReader(TDengineSourceConfig config, SourceReader.Context readerContext) {
6359
this.config = config;
64-
this.sourceSplits = Sets.newHashSet();
60+
this.sourceSplits = new ConcurrentLinkedDeque<>();
6561
this.context = readerContext;
6662
}
6763

6864
@Override
6965
public void pollNext(Collector<SeaTunnelRow> collector) throws InterruptedException {
70-
if (sourceSplits.isEmpty()) {
71-
Thread.sleep(THREAD_WAIT_TIME);
72-
return;
73-
}
7466
synchronized (collector.getCheckpointLock()) {
75-
sourceSplits.forEach(
76-
split -> {
77-
try {
78-
read(split, collector);
79-
} catch (Exception e) {
80-
throw new TDengineConnectorException(
81-
CommonErrorCodeDeprecated.READER_OPERATION_FAILED,
82-
"TDengine split read error",
83-
e);
84-
}
85-
});
86-
}
87-
88-
if (Boundedness.BOUNDED.equals(context.getBoundedness())) {
89-
// signal to the source that we have reached the end of the data.
90-
log.info("Closed the bounded TDengine source");
91-
context.signalNoMoreElement();
67+
log.info("polling new split from queue!");
68+
TDengineSourceSplit split = sourceSplits.poll();
69+
if (Objects.nonNull(split)) {
70+
log.info(
71+
"starting run new split {}, query sql: {}!",
72+
split.splitId(),
73+
split.getQuery());
74+
try {
75+
read(split, collector);
76+
} catch (Exception e) {
77+
throw new TDengineConnectorException(
78+
CommonErrorCodeDeprecated.READER_OPERATION_FAILED,
79+
"TDengine split read error",
80+
e);
81+
}
82+
} else if (noMoreSplit && sourceSplits.isEmpty()) {
83+
// signal to the source that we have reached the end of the data.
84+
log.info("Closed the bounded TDengine source");
85+
context.signalNoMoreElement();
86+
} else {
87+
Thread.sleep(1000L);
88+
}
9289
}
9390
}
9491

9592
@Override
9693
public void open() {
97-
String jdbcUrl =
98-
StringUtils.join(
99-
config.getUrl(),
100-
config.getDatabase(),
101-
"?user=",
102-
config.getUsername(),
103-
"&password=",
104-
config.getPassword());
105-
Properties connProps = new Properties();
106-
// todo: when TSDBDriver.PROPERTY_KEY_BATCH_LOAD set to "true",
107-
// there is a exception : Caused by: java.sql.SQLException: can't create connection with
108-
// server
109-
// under docker network env
110-
// @bobo (tdengine)
111-
connProps.setProperty(TSDBDriver.PROPERTY_KEY_BATCH_LOAD, "false");
94+
String jdbcUrl = config.getUrl();
95+
96+
Properties properties = new Properties();
97+
properties.put(TSDBDriver.PROPERTY_KEY_USER, config.getUsername());
98+
properties.put(TSDBDriver.PROPERTY_KEY_PASSWORD, config.getPassword());
99+
112100
try {
113-
// check td driver whether exist and if not, try to register
114101
checkDriverExist(jdbcUrl);
115-
conn = DriverManager.getConnection(jdbcUrl, connProps);
102+
conn = DriverManager.getConnection(jdbcUrl, properties);
116103
} catch (SQLException e) {
117104
throw new TDengineConnectorException(
118105
CommonErrorCodeDeprecated.READER_OPERATION_FAILED,
119-
"get TDengine connection failed:" + jdbcUrl);
106+
"get TDengine connection failed:" + jdbcUrl,
107+
e);
120108
}
121109
}
122110

@@ -135,8 +123,8 @@ public void close() {
135123
}
136124

137125
private void read(TDengineSourceSplit split, Collector<SeaTunnelRow> output) throws Exception {
138-
try (Statement statement = conn.createStatement()) {
139-
final ResultSet resultSet = statement.executeQuery(split.getQuery());
126+
try (Statement statement = conn.createStatement();
127+
ResultSet resultSet = statement.executeQuery(split.getQuery())) {
140128
ResultSetMetaData meta = resultSet.getMetaData();
141129

142130
while (resultSet.next()) {
@@ -151,6 +139,8 @@ private void read(TDengineSourceSplit split, Collector<SeaTunnelRow> output) thr
151139
}
152140

153141
private Object convertDataType(Object object) {
142+
if (Objects.isNull(object)) return null;
143+
154144
if (Timestamp.class.equals(object.getClass())) {
155145
return ((Timestamp) object).toLocalDateTime();
156146
} else if (byte[].class.equals(object.getClass())) {
@@ -171,7 +161,8 @@ public void addSplits(List<TDengineSourceSplit> splits) {
171161

172162
@Override
173163
public void handleNoMoreSplits() {
174-
// do nothing
164+
log.info("no more split accepted!");
165+
noMoreSplit = true;
175166
}
176167

177168
@Override

0 commit comments

Comments
 (0)