Skip to content

Commit cfae514

Browse files
committed
Merge branch 'main' into v2_product_name
2 parents 0da5e0d + ce65071 commit cfae514

File tree

48 files changed

+1753
-958
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

48 files changed

+1753
-958
lines changed

clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseClientOption.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -218,6 +218,8 @@ public enum ClickHouseClientOption implements ClickHouseOption {
218218
MAX_RESULT_ROWS("max_result_rows", 0L,
219219
"Limit on the number of rows in the result. "
220220
+ "Also checked for subqueries, and on remote servers when running parts of a distributed query."),
221+
222+
RESULT_OVERFLOW_MODE("result_overflow_mode", "throw","What to do if the result is overflowed."),
221223
/**
222224
* Maximum size of thread pool for each client.
223225
*/

clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnection.java

Lines changed: 25 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -118,10 +118,25 @@ static String buildQueryParams(ClickHouseRequest<?> request, Map<String, Seriali
118118
appendQueryParameter(builder, settingKey, String.valueOf(config.getMaxExecutionTime()));
119119
}
120120
settingKey = ClickHouseClientOption.MAX_RESULT_ROWS.getKey();
121-
if (config.getMaxResultRows() > 0L && !settings.containsKey(settingKey)) {
121+
boolean hasRequestSetting = settings.containsKey(settingKey);
122+
if (config.getMaxResultRows() > 0L && !hasRequestSetting) {
123+
// set on client level
122124
appendQueryParameter(builder, settingKey, String.valueOf(config.getMaxResultRows()));
123-
appendQueryParameter(builder, "result_overflow_mode", "break");
125+
} else if (hasRequestSetting) {
126+
// set on request level
127+
Object value = settings.get(settingKey);
128+
if (value instanceof Number && ((Number) value).longValue() > 0L) {
129+
appendQueryParameter(builder, settingKey, String.valueOf(value));
130+
} else if (value instanceof String && !(((String) value).isEmpty() || "0".equals(value))) {
131+
appendQueryParameter(builder, settingKey, (String) value);
132+
}
133+
}
134+
135+
if (config.hasOption(ClickHouseClientOption.RESULT_OVERFLOW_MODE)) {
136+
appendQueryParameter(builder, ClickHouseClientOption.RESULT_OVERFLOW_MODE.getKey(),
137+
config.getStrOption(ClickHouseClientOption.RESULT_OVERFLOW_MODE));
124138
}
139+
125140
settingKey = "log_comment";
126141
if (!stmts.isEmpty() && config.getBoolOption(ClickHouseClientOption.LOG_LEADING_COMMENT)
127142
&& !settings.containsKey(settingKey)) {
@@ -162,11 +177,9 @@ static String buildQueryParams(ClickHouseRequest<?> request, Map<String, Seriali
162177
}
163178

164179
for (Entry<String, Serializable> entry : settings.entrySet()) {
165-
// Skip internal settings
166-
if (entry.getKey().equalsIgnoreCase("_set_roles_stmt")) {
167-
continue;
180+
if (!processedSettings.contains(entry.getKey())) {
181+
appendQueryParameter(builder, entry.getKey(), String.valueOf(entry.getValue()));
168182
}
169-
appendQueryParameter(builder, entry.getKey(), String.valueOf(entry.getValue()));
170183
}
171184

172185
if (builder.length() > 0) {
@@ -175,6 +188,12 @@ static String buildQueryParams(ClickHouseRequest<?> request, Map<String, Seriali
175188
return builder.toString();
176189
}
177190

191+
// Settings that are processed by the client and appropriate parameters are set
192+
private static final HashSet<String> processedSettings = new HashSet<>(Arrays.asList(
193+
"_set_roles_stmt",
194+
ClickHouseClientOption.MAX_RESULT_ROWS.getKey()
195+
));
196+
178197
static String buildUrl(String baseUrl, ClickHouseRequest<?> request, Map<String, Serializable> additionalParams) {
179198
StringBuilder builder = new StringBuilder().append(baseUrl);
180199
// TODO: Using default until we will remove
Lines changed: 22 additions & 55 deletions
Original file line numberDiff line numberDiff line change
@@ -1,97 +1,64 @@
11
package com.clickhouse.jdbc;
22

33
import javax.sql.DataSource;
4-
5-
import com.clickhouse.client.config.ClickHouseDefaults;
6-
import com.clickhouse.jdbc.internal.ClickHouseConnectionImpl;
7-
import com.clickhouse.jdbc.internal.ClickHouseJdbcUrlParser;
8-
import com.clickhouse.jdbc.internal.ClickHouseJdbcUrlParser.ConnectionInfo;
9-
104
import java.io.PrintWriter;
115
import java.sql.Connection;
12-
import java.sql.DriverManager;
136
import java.sql.SQLException;
147
import java.sql.SQLFeatureNotSupportedException;
158
import java.util.Properties;
16-
import java.util.logging.Logger;
17-
18-
public class ClickHouseDataSource extends JdbcWrapper implements DataSource {
19-
private final String url;
20-
private final Properties props;
21-
22-
protected final ClickHouseDriver driver;
23-
protected final ConnectionInfo connInfo;
249

25-
protected PrintWriter printWriter;
26-
protected int loginTimeoutSeconds = 0;
10+
public class ClickHouseDataSource implements javax.sql.DataSource, com.clickhouse.jdbc.JdbcV2Wrapper {
11+
private final DataSource dataSource;
12+
private final ClickHouseDriver driver;
2713

2814
public ClickHouseDataSource(String url) throws SQLException {
2915
this(url, new Properties());
3016
}
3117

3218
public ClickHouseDataSource(String url, Properties properties) throws SQLException {
33-
if (url == null) {
34-
throw new IllegalArgumentException("Incorrect ClickHouse jdbc url. It must be not null");
35-
}
36-
this.url = url;
37-
this.props = new Properties();
38-
if (properties != null && !properties.isEmpty()) {
39-
this.props.putAll(properties);
40-
}
41-
4219
this.driver = new ClickHouseDriver();
43-
new VersionSelectingDriver();//This is a workaround to make sure the driver is loaded
44-
this.connInfo = ClickHouseJdbcUrlParser.parse(url, properties);
20+
21+
if (driver.isV2(url)) {
22+
//v2
23+
this.dataSource = new com.clickhouse.jdbc.DataSourceImpl(url, properties);
24+
} else {
25+
//v1
26+
this.dataSource = new DataSourceV1(url, properties);
27+
}
4528
}
4629

4730
@Override
48-
public ClickHouseConnection getConnection() throws SQLException {
49-
return new ClickHouseConnectionImpl(connInfo);
31+
public Connection getConnection() throws SQLException {
32+
return dataSource.getConnection();
5033
}
5134

5235
@Override
53-
public ClickHouseConnection getConnection(String username, String password) throws SQLException {
54-
if (username == null || username.isEmpty()) {
55-
throw SqlExceptionUtils.clientError("Non-empty user name is required");
56-
}
57-
58-
if (password == null) {
59-
password = "";
60-
}
61-
62-
if (username.equals(props.getProperty(ClickHouseDefaults.USER.getKey()))
63-
&& password.equals(props.getProperty(ClickHouseDefaults.PASSWORD.getKey()))) {
64-
return new ClickHouseConnectionImpl(connInfo);
65-
}
66-
67-
Properties properties = new Properties();
68-
properties.putAll(this.props);
69-
properties.setProperty(ClickHouseDefaults.USER.getKey(), username);
70-
properties.setProperty(ClickHouseDefaults.PASSWORD.getKey(), password);
71-
return new ClickHouseConnectionImpl(url, properties);
36+
public Connection getConnection(String username, String password) throws SQLException {
37+
return dataSource.getConnection(username, password);
7238
}
7339

7440
@Override
7541
public PrintWriter getLogWriter() throws SQLException {
76-
return printWriter;
42+
return dataSource.getLogWriter();
7743
}
7844

7945
@Override
8046
public void setLogWriter(PrintWriter out) throws SQLException {
81-
printWriter = out;
47+
dataSource.setLogWriter(out);
8248
}
8349

8450
@Override
8551
public void setLoginTimeout(int seconds) throws SQLException {
86-
loginTimeoutSeconds = seconds;
52+
dataSource.setLoginTimeout(seconds);
8753
}
8854

8955
@Override
9056
public int getLoginTimeout() throws SQLException {
91-
return loginTimeoutSeconds;
57+
return dataSource.getLoginTimeout();
9258
}
9359

94-
public Logger getParentLogger() throws SQLFeatureNotSupportedException {
95-
return ClickHouseDriver.parentLogger;
60+
@Override
61+
public java.util.logging.Logger getParentLogger() throws SQLFeatureNotSupportedException {
62+
return dataSource.getParentLogger();
9663
}
9764
}

clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaData.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -149,17 +149,17 @@ public String getDriverName() throws SQLException {
149149

150150
@Override
151151
public String getDriverVersion() throws SQLException {
152-
return ClickHouseDriver.driverVersionString;
152+
return DriverV1.driverVersionString;
153153
}
154154

155155
@Override
156156
public int getDriverMajorVersion() {
157-
return ClickHouseDriver.driverVersion.getMajorVersion();
157+
return DriverV1.driverVersion.getMajorVersion();
158158
}
159159

160160
@Override
161161
public int getDriverMinorVersion() {
162-
return ClickHouseDriver.driverVersion.getMinorVersion();
162+
return DriverV1.driverVersion.getMinorVersion();
163163
}
164164

165165
@Override
@@ -1217,12 +1217,12 @@ public int getDatabaseMinorVersion() throws SQLException {
12171217

12181218
@Override
12191219
public int getJDBCMajorVersion() throws SQLException {
1220-
return ClickHouseDriver.specVersion.getMajorVersion();
1220+
return DriverV1.specVersion.getMajorVersion();
12211221
}
12221222

12231223
@Override
12241224
public int getJDBCMinorVersion() throws SQLException {
1225-
return ClickHouseDriver.specVersion.getMinorVersion();
1225+
return DriverV1.specVersion.getMinorVersion();
12261226
}
12271227

12281228
@Override

0 commit comments

Comments
 (0)