Skip to content

Commit 7e34551

Browse files
authored
[Feat] Append the url's database and parameters to newly created connection #142
[Enhance] Append the url's database and parameters to newly created connection #142
2 parents 279b6ac + 6f409ba commit 7e34551

File tree

4 files changed

+71
-27
lines changed

4 files changed

+71
-27
lines changed

flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/catalog/ClickHouseCatalog.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -132,7 +132,7 @@ public ClickHouseCatalog(
132132
checkArgument(!isNullOrWhitespaceOnly(username), "username cannot be null or empty");
133133
checkArgument(!isNullOrWhitespaceOnly(password), "password cannot be null or empty");
134134

135-
this.baseUrl = baseUrl.endsWith("/") ? baseUrl : baseUrl + "/";
135+
this.baseUrl = baseUrl;
136136
this.username = username;
137137
this.password = password;
138138
this.ignorePrimaryKey =

flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/connection/ClickHouseConnectionProvider.java

Lines changed: 17 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -28,8 +28,6 @@
2828
import org.slf4j.LoggerFactory;
2929

3030
import java.io.Serializable;
31-
import java.sql.PreparedStatement;
32-
import java.sql.ResultSet;
3331
import java.sql.SQLException;
3432
import java.util.ArrayList;
3533
import java.util.HashMap;
@@ -38,7 +36,7 @@
3836
import java.util.Properties;
3937

4038
import static java.util.stream.Collectors.toList;
41-
import static org.apache.flink.connector.clickhouse.util.ClickHouseJdbcUtil.getActualHttpPort;
39+
import static org.apache.flink.connector.clickhouse.util.ClickHouseJdbcUtil.getClusterSpec;
4240

4341
/** ClickHouse connection provider. Use ClickHouseDriver to create a connection. */
4442
public class ClickHouseConnectionProvider implements Serializable {
@@ -47,9 +45,6 @@ public class ClickHouseConnectionProvider implements Serializable {
4745

4846
private static final Logger LOG = LoggerFactory.getLogger(ClickHouseConnectionProvider.class);
4947

50-
private static final String QUERY_CLUSTER_INFO_SQL =
51-
"SELECT shard_num, host_address, port FROM system.clusters WHERE cluster = ? ORDER BY shard_num, replica_num ASC";
52-
5348
private final ClickHouseConnectionOptions options;
5449

5550
private final Properties connectionProperties;
@@ -74,17 +69,19 @@ public boolean isConnectionValid() throws SQLException {
7469

7570
public synchronized ClickHouseConnection getOrCreateConnection() throws SQLException {
7671
if (connection == null) {
77-
connection = createConnection(options.getUrl(), options.getDatabaseName());
72+
connection = createConnection(options.getUrl());
7873
}
7974
return connection;
8075
}
8176

8277
public synchronized Map<Integer, ClickHouseConnection> createShardConnections(
8378
ClusterSpec clusterSpec, String defaultDatabase) throws SQLException {
8479
Map<Integer, ClickHouseConnection> connectionMap = new HashMap<>();
80+
String urlSuffix = options.getUrlSuffix();
8581
for (ShardSpec shardSpec : clusterSpec.getShards()) {
82+
String shardUrl = shardSpec.getJdbcUrls() + urlSuffix;
8683
ClickHouseConnection connection =
87-
createAndStoreShardConnection(shardSpec.getJdbcUrls(), defaultDatabase);
84+
createAndStoreShardConnection(shardUrl, defaultDatabase);
8885
connectionMap.put(shardSpec.getNum(), connection);
8986
}
9087

@@ -97,35 +94,29 @@ public synchronized ClickHouseConnection createAndStoreShardConnection(
9794
shardConnections = new ArrayList<>();
9895
}
9996

100-
ClickHouseConnection connection = createConnection(url, database);
97+
ClickHouseConnection connection = createConnection(url);
10198
shardConnections.add(connection);
10299
return connection;
103100
}
104101

105102
public List<String> getShardUrls(String remoteCluster) throws SQLException {
106-
Map<Long, List<String>> shardsMap = new HashMap<>();
103+
Map<Integer, String> shardsMap = new HashMap<>();
107104
ClickHouseConnection conn = getOrCreateConnection();
108-
try (PreparedStatement stmt = conn.prepareStatement(QUERY_CLUSTER_INFO_SQL)) {
109-
stmt.setString(1, remoteCluster);
110-
try (ResultSet rs = stmt.executeQuery()) {
111-
while (rs.next()) {
112-
String host = rs.getString("host_address");
113-
int port = getActualHttpPort(host, rs.getInt("port"));
114-
List<String> shardUrls =
115-
shardsMap.computeIfAbsent(
116-
rs.getLong("shard_num"), k -> new ArrayList<>());
117-
shardUrls.add(host + ":" + port);
118-
}
119-
}
105+
ClusterSpec clusterSpec = getClusterSpec(conn, remoteCluster);
106+
String urlSuffix = options.getUrlSuffix();
107+
for (ShardSpec shardSpec : clusterSpec.getShards()) {
108+
String shardUrl = shardSpec.getJdbcUrls() + urlSuffix;
109+
shardsMap.put(shardSpec.getNum(), shardUrl);
120110
}
121111

122-
return shardsMap.values().stream()
123-
.map(urls -> "jdbc:ch://" + String.join(",", urls))
112+
return shardsMap.entrySet().stream()
113+
.sorted(Map.Entry.comparingByKey())
114+
.map(Map.Entry::getValue)
124115
.collect(toList());
125116
}
126117

127-
private ClickHouseConnection createConnection(String url, String database) throws SQLException {
128-
LOG.info("connecting to {}, database {}", url, database);
118+
private ClickHouseConnection createConnection(String url) throws SQLException {
119+
LOG.info("connecting to {}", url);
129120
Properties configuration = new Properties();
130121
configuration.putAll(connectionProperties);
131122
if (options.getUsername().isPresent()) {

flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/options/ClickHouseConnectionOptions.java

Lines changed: 33 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,16 +17,26 @@
1717

1818
package org.apache.flink.connector.clickhouse.internal.options;
1919

20+
import org.apache.flink.annotation.VisibleForTesting;
21+
2022
import javax.annotation.Nullable;
2123

2224
import java.io.Serializable;
2325
import java.util.Optional;
26+
import java.util.regex.Matcher;
27+
import java.util.regex.Pattern;
28+
29+
import static org.apache.flink.connector.clickhouse.util.ClickHouseUtil.EMPTY;
30+
import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
2431

2532
/** ClickHouse connection options. */
2633
public class ClickHouseConnectionOptions implements Serializable {
2734

2835
private static final long serialVersionUID = 1L;
2936

37+
public static final Pattern URL_PATTERN =
38+
Pattern.compile("[^/]+//[^/?]+(/(?<database>[^?]*))?(\\?(?<param>\\S+))?");
39+
3040
private final String url;
3141

3242
private final String username;
@@ -37,6 +47,12 @@ public class ClickHouseConnectionOptions implements Serializable {
3747

3848
private final String tableName;
3949

50+
// For testing.
51+
@VisibleForTesting
52+
public ClickHouseConnectionOptions(String url) {
53+
this(url, null, null, null, null);
54+
}
55+
4056
protected ClickHouseConnectionOptions(
4157
String url,
4258
@Nullable String username,
@@ -50,6 +66,23 @@ protected ClickHouseConnectionOptions(
5066
this.tableName = tableName;
5167
}
5268

69+
/**
70+
* The format of the URL suffix is as follows: {@code
71+
* [/<database>][?param1=value1&param2=value2]}.
72+
*/
73+
public String getUrlSuffix() {
74+
Matcher matcher = URL_PATTERN.matcher(url);
75+
if (!matcher.find()) {
76+
return EMPTY;
77+
}
78+
79+
String database = matcher.group("database");
80+
String param = matcher.group("param");
81+
database = isNullOrWhitespaceOnly(database) ? EMPTY : "/" + database;
82+
param = isNullOrWhitespaceOnly(param) ? EMPTY : "?" + param;
83+
return database + param;
84+
}
85+
5386
public String getUrl() {
5487
return this.url;
5588
}

flink-connector-clickhouse/src/test/java/org/apache/flink/connector/clickhouse/AppTest.java

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
package org.apache.flink.connector.clickhouse;
1919

20+
import org.apache.flink.connector.clickhouse.internal.options.ClickHouseConnectionOptions;
2021
import org.apache.flink.connector.clickhouse.internal.partitioner.ValuePartitioner;
2122
import org.apache.flink.connector.clickhouse.internal.schema.ClusterSpec;
2223
import org.apache.flink.connector.clickhouse.internal.schema.ShardSpec;
@@ -218,4 +219,23 @@ public void parseEngineFullTest() {
218219
requireNonNull(parseShardingKey(matcher.group("shardingKey"))).explain());
219220
}
220221
}
222+
223+
@Test
224+
public void parseJdbcUriTest() {
225+
String[] urls = {
226+
"jdbc:ch://localhost:8123",
227+
"jdbc:ch://localhost:8123?",
228+
"jdbc:ch://localhost:8123?ssl=true&sslmode=STRICT",
229+
"jdbc:ch://localhost:8123/",
230+
"jdbc:ch://localhost:8123/?ssl=true&sslmode=STRICT",
231+
"jdbc:ch://localhost:8123/default?ssl=true&sslmode=STRICT",
232+
"jdbc:ch://localhost:8123,127.0.0.1:8123/default?ssl=true&sslmode=STRICT"
233+
};
234+
235+
for (String url : urls) {
236+
String urlSuffix = new ClickHouseConnectionOptions(url).getUrlSuffix();
237+
String urlPrefix = url.substring(0, url.lastIndexOf(urlSuffix));
238+
assertEquals(url, urlPrefix + urlSuffix);
239+
}
240+
}
221241
}

0 commit comments

Comments
 (0)