Skip to content

Commit b9d1ca0

Browse files
committed
Merge remote-tracking branch 'origin/main'
2 parents 8d77e8f + 49d5322 commit b9d1ca0

File tree

7 files changed

+268
-46
lines changed

7 files changed

+268
-46
lines changed

client-v2/src/main/java/com/clickhouse/client/api/Client.java

Lines changed: 31 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -192,16 +192,24 @@ private Client(Set<String> endpoints, Map<String,String> configuration, boolean
192192
*
193193
*/
194194
public void loadServerInfo() {
195-
try (QueryResponse response = this.query("SELECT currentUser() AS user, timezone() AS timezone, version() AS version LIMIT 1").get()) {
196-
try (ClickHouseBinaryFormatReader reader = this.newBinaryFormatReader(response)) {
197-
if (reader.next() != null) {
198-
this.configuration.put(ClientConfigProperties.USER.getKey(), reader.getString("user"));
199-
this.configuration.put(ClientConfigProperties.SERVER_TIMEZONE.getKey(), reader.getString("timezone"));
200-
serverVersion = reader.getString("version");
195+
// only if 2 properties are set disable retrieval from server
196+
if (!this.configuration.containsKey(ClientConfigProperties.SERVER_TIMEZONE.getKey()) && !this.configuration.containsKey(ClientConfigProperties.SERVER_VERSION.getKey())) {
197+
try (QueryResponse response = this.query("SELECT currentUser() AS user, timezone() AS timezone, version() AS version LIMIT 1").get()) {
198+
try (ClickHouseBinaryFormatReader reader = this.newBinaryFormatReader(response)) {
199+
if (reader.next() != null) {
200+
this.configuration.put(ClientConfigProperties.USER.getKey(), reader.getString("user"));
201+
this.configuration.put(ClientConfigProperties.SERVER_TIMEZONE.getKey(), reader.getString("timezone"));
202+
serverVersion = reader.getString("version");
203+
}
201204
}
205+
} catch (Exception e) {
206+
throw new ClientException("Failed to get server info", e);
207+
}
208+
} else {
209+
LOG.info("Using server version " + this.configuration.get(ClientConfigProperties.SERVER_VERSION.getKey()) + " and timezone " + this.configuration.get(ClientConfigProperties.SERVER_TIMEZONE.getKey()) );
210+
if (this.configuration.containsKey(ClientConfigProperties.SERVER_VERSION.getKey())) {
211+
serverVersion = this.configuration.get(ClientConfigProperties.SERVER_VERSION.getKey());
202212
}
203-
} catch (Exception e) {
204-
throw new ClientException("Failed to get server info", e);
205213
}
206214
}
207215

@@ -991,6 +999,17 @@ public Builder registerClientMetrics(Object registry, String name) {
991999
return this;
9921000
}
9931001

1002+
/**
1003+
* Sets server version that the client is interacting with.
1004+
*
1005+
* @param serverVersion - ClickHouse server version
1006+
* @return same instance of the builder
1007+
*/
1008+
public Builder setServerVersion(String serverVersion) {
1009+
this.configuration.put(ClientConfigProperties.SERVER_VERSION.getKey(), serverVersion);
1010+
return this;
1011+
}
1012+
9941013
public Client build() {
9951014
setDefaults();
9961015

@@ -2174,6 +2193,10 @@ public String getServerVersion() {
21742193
return this.serverVersion;
21752194
}
21762195

2196+
public String getServerTimeZone() {
2197+
return this.configuration.get(ClientConfigProperties.SERVER_TIMEZONE.getKey());
2198+
}
2199+
21772200
public String getClientVersion() {
21782201
return clientVersion;
21792202
}

client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,8 @@ public enum ClientConfigProperties {
3838

3939
USE_TIMEZONE("use_time_zone"),
4040

41+
SERVER_VERSION("server_version"),
42+
4143
SERVER_TIMEZONE("server_time_zone"),
4244

4345
ASYNC_OPERATIONS("async"),

jdbc-v2/src/main/java/com/clickhouse/jdbc/PreparedStatementImpl.java

Lines changed: 137 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,7 @@
11
package com.clickhouse.jdbc;
22

33
import com.clickhouse.client.api.metadata.TableSchema;
4+
import com.clickhouse.client.api.query.QuerySettings;
45
import com.clickhouse.data.Tuple;
56
import com.clickhouse.jdbc.internal.ExceptionUtils;
67
import com.clickhouse.jdbc.internal.JdbcUtils;
@@ -101,13 +102,14 @@ private String compileSql(String []segments) {
101102
@Override
102103
public ResultSet executeQuery() throws SQLException {
103104
checkClosed();
104-
return executeQuery(compileSql(sqlSegments));
105+
return super.executeQueryImpl(compileSql(sqlSegments), new QuerySettings().setDatabase(connection.getSchema()));
105106
}
106107

107108
@Override
108109
public int executeUpdate() throws SQLException {
109110
checkClosed();
110-
return executeUpdate(compileSql(sqlSegments));
111+
return super.executeUpdateImpl(compileSql(sqlSegments), statementType,
112+
new QuerySettings().setDatabase(connection.getSchema()));
111113
}
112114

113115
@Override
@@ -234,17 +236,18 @@ public void setObject(int parameterIndex, Object x) throws SQLException {
234236
@Override
235237
public boolean execute() throws SQLException {
236238
checkClosed();
237-
return execute(compileSql(sqlSegments));
239+
return super.executeImpl(compileSql(sqlSegments), statementType,
240+
new QuerySettings().setDatabase(connection.getSchema()));
238241
}
239242

240243
@Override
241244
public void addBatch() throws SQLException {
242245
checkClosed();
243246
if (statementType == StatementType.INSERT) {
244247
// adding values to the end of big INSERT statement.
245-
addBatch(compileSql(valueSegments));
248+
super.addBatch(compileSql(valueSegments));
246249
} else {
247-
addBatch(compileSql(sqlSegments));
250+
super.addBatch(compileSql(sqlSegments));
248251
}
249252
}
250253

@@ -259,7 +262,8 @@ public int[] executeBatch() throws SQLException {
259262
sb.append(sql).append(",");
260263
}
261264
sb.setCharAt(sb.length() - 1, ';');
262-
int rowsInserted = executeUpdate(sb.toString());
265+
int rowsInserted = executeUpdateImpl(sb.toString(), statementType,
266+
new QuerySettings().setDatabase(connection.getSchema()));
263267
// clear batch and re-add insert into
264268
int[] results = new int[batch.size()];
265269
if (rowsInserted == batch.size()) {
@@ -274,18 +278,22 @@ public int[] executeBatch() throws SQLException {
274278
return results;
275279
} else {
276280
// run executeBatch
277-
return super.executeBatch();
281+
return executeBatchImpl().stream().mapToInt(Integer::intValue).toArray();
278282
}
279283
}
280284

281285
@Override
282286
public long[] executeLargeBatch() throws SQLException {
283-
int[] results = executeBatch();
284-
long[] longResults = new long[results.length];
285-
for (int i = 0; i < results.length; i++) {
286-
longResults[i] = results[i];
287+
return executeBatchImpl().stream().mapToLong(Integer::longValue).toArray();
288+
}
289+
290+
private List<Integer> executeBatchImpl() throws SQLException {
291+
List<Integer> results = new ArrayList<>();
292+
QuerySettings settings = new QuerySettings().setDatabase(connection.getSchema());
293+
for (String sql : batch) {
294+
results.add(executeUpdateImpl(sql, statementType, settings));
287295
}
288-
return longResults;
296+
return results;
289297
}
290298

291299
@Override
@@ -412,7 +420,8 @@ public ParameterMetaData getParameterMetaData() throws SQLException {
412420
@Override
413421
public void setRowId(int parameterIndex, RowId x) throws SQLException {
414422
checkClosed();
415-
parameters[parameterIndex - 1] = encodeObject(x);
423+
throw new SQLException("ROWID type is not supported by ClickHouse.",
424+
ExceptionUtils.SQL_STATE_FEATURE_NOT_SUPPORTED);
416425
}
417426

418427
@Override
@@ -540,6 +549,118 @@ public long executeLargeUpdate() throws SQLException {
540549
return executeUpdate();
541550
}
542551

552+
@Override
553+
public final void addBatch(String sql) throws SQLException {
554+
checkClosed();
555+
throw new SQLException(
556+
"addBatch(String) cannot be called in PreparedStatement or CallableStatement!",
557+
ExceptionUtils.SQL_STATE_WRONG_OBJECT_TYPE);
558+
}
559+
560+
@Override
561+
public final boolean execute(String sql) throws SQLException {
562+
checkClosed();
563+
throw new SQLException(
564+
"execute(String) cannot be called in PreparedStatement or CallableStatement!",
565+
ExceptionUtils.SQL_STATE_WRONG_OBJECT_TYPE);
566+
}
567+
568+
@Override
569+
public final boolean execute(String sql, int autoGeneratedKeys) throws SQLException {
570+
checkClosed();
571+
throw new SQLException(
572+
"execute(String, int) cannot be called in PreparedStatement or CallableStatement!",
573+
ExceptionUtils.SQL_STATE_WRONG_OBJECT_TYPE);
574+
}
575+
576+
@Override
577+
public final boolean execute(String sql, int[] columnIndexes) throws SQLException {
578+
checkClosed();
579+
throw new SQLException(
580+
"execute(String, int[]) cannot be called in PreparedStatement or CallableStatement!",
581+
ExceptionUtils.SQL_STATE_WRONG_OBJECT_TYPE);
582+
}
583+
584+
@Override
585+
public final boolean execute(String sql, String[] columnNames) throws SQLException {
586+
checkClosed();
587+
throw new SQLException(
588+
"execute(String, String[]) cannot be called in PreparedStatement or CallableStatement!",
589+
ExceptionUtils.SQL_STATE_WRONG_OBJECT_TYPE);
590+
}
591+
592+
@Override
593+
public final long executeLargeUpdate(String sql) throws SQLException {
594+
checkClosed();
595+
throw new SQLException(
596+
"executeLargeUpdate(String) cannot be called in PreparedStatement or CallableStatement!",
597+
ExceptionUtils.SQL_STATE_WRONG_OBJECT_TYPE);
598+
}
599+
600+
@Override
601+
public final long executeLargeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
602+
checkClosed();
603+
throw new SQLException(
604+
"executeLargeUpdate(String, int) cannot be called in PreparedStatement or CallableStatement!",
605+
ExceptionUtils.SQL_STATE_WRONG_OBJECT_TYPE);
606+
}
607+
608+
@Override
609+
public final long executeLargeUpdate(String sql, int[] columnIndexes) throws SQLException {
610+
checkClosed();
611+
throw new SQLException(
612+
"executeLargeUpdate(String, int[]) cannot be called in PreparedStatement or CallableStatement!",
613+
ExceptionUtils.SQL_STATE_WRONG_OBJECT_TYPE);
614+
}
615+
616+
@Override
617+
public final long executeLargeUpdate(String sql, String[] columnNames) throws SQLException {
618+
checkClosed();
619+
throw new SQLException(
620+
"executeLargeUpdate(String, String[]) cannot be called in PreparedStatement or CallableStatement!",
621+
ExceptionUtils.SQL_STATE_WRONG_OBJECT_TYPE);
622+
}
623+
624+
@Override
625+
public final ResultSet executeQuery(String sql) throws SQLException {
626+
checkClosed();
627+
throw new SQLException(
628+
"executeQuery(String) cannot be called in PreparedStatement or CallableStatement!",
629+
ExceptionUtils.SQL_STATE_WRONG_OBJECT_TYPE);
630+
}
631+
632+
@Override
633+
public final int executeUpdate(String sql) throws SQLException {
634+
checkClosed();
635+
throw new SQLException(
636+
"executeUpdate(String) cannot be called in PreparedStatement or CallableStatement!",
637+
ExceptionUtils.SQL_STATE_WRONG_OBJECT_TYPE);
638+
}
639+
640+
@Override
641+
public final int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
642+
checkClosed();
643+
throw new SQLException(
644+
"executeUpdate(String, int) cannot be called in PreparedStatement or CallableStatement!",
645+
ExceptionUtils.SQL_STATE_WRONG_OBJECT_TYPE);
646+
}
647+
648+
@Override
649+
public final int executeUpdate(String sql, int[] columnIndexes) throws SQLException {
650+
checkClosed();
651+
throw new SQLException(
652+
"executeUpdate(String, int[]) cannot be called in PreparedStatement or CallableStatement!",
653+
ExceptionUtils.SQL_STATE_WRONG_OBJECT_TYPE);
654+
}
655+
656+
@Override
657+
public final int executeUpdate(String sql, String[] columnNames) throws SQLException {
658+
checkClosed();
659+
throw new SQLException(
660+
"executeUpdate(String, String[]) cannot be called in PreparedStatement or CallableStatement!",
661+
ExceptionUtils.SQL_STATE_WRONG_OBJECT_TYPE);
662+
}
663+
543664
private static String encodeObject(Object x) throws SQLException {
544665
LOG.trace("Encoding object: {}", x);
545666

@@ -590,7 +711,9 @@ private static String encodeObject(Object x) throws SQLException {
590711
for (Object item : (Collection<?>) x) {
591712
listString.append(encodeObject(item)).append(", ");
592713
}
593-
listString.delete(listString.length() - 2, listString.length());
714+
if (listString.length() > 1) {
715+
listString.delete(listString.length() - 2, listString.length());
716+
}
594717
listString.append("]");
595718

596719
return listString.toString();

jdbc-v2/src/main/java/com/clickhouse/jdbc/StatementImpl.java

Lines changed: 12 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -56,7 +56,7 @@ protected void checkClosed() throws SQLException {
5656
}
5757
}
5858

59-
protected enum StatementType {
59+
public enum StatementType {
6060
SELECT, INSERT, DELETE, UPDATE, CREATE, DROP, ALTER, TRUNCATE, USE, SHOW, DESCRIBE, EXPLAIN, SET, KILL, OTHER, INSERT_INTO_SELECT
6161
}
6262

@@ -148,7 +148,7 @@ protected String getLastSql() {
148148
@Override
149149
public ResultSet executeQuery(String sql) throws SQLException {
150150
checkClosed();
151-
return executeQuery(sql, new QuerySettings().setDatabase(schema));
151+
return executeQueryImpl(sql, new QuerySettings().setDatabase(schema));
152152
}
153153

154154
private void closePreviousResultSet() {
@@ -165,7 +165,7 @@ private void closePreviousResultSet() {
165165
}
166166
}
167167

168-
public ResultSetImpl executeQuery(String sql, QuerySettings settings) throws SQLException {
168+
public ResultSetImpl executeQueryImpl(String sql, QuerySettings settings) throws SQLException {
169169
checkClosed();
170170
// Closing before trying to do next request. Otherwise, deadlock because previous connection will not be
171171
// release before this one completes.
@@ -213,13 +213,12 @@ public ResultSetImpl executeQuery(String sql, QuerySettings settings) throws SQL
213213
@Override
214214
public int executeUpdate(String sql) throws SQLException {
215215
checkClosed();
216-
return executeUpdate(sql, new QuerySettings().setDatabase(schema));
216+
return executeUpdateImpl(sql, parseStatementType(sql), new QuerySettings().setDatabase(schema));
217217
}
218218

219-
public int executeUpdate(String sql, QuerySettings settings) throws SQLException {
220-
// TODO: close current result set?
219+
protected int executeUpdateImpl(String sql, StatementType type, QuerySettings settings) throws SQLException {
221220
checkClosed();
222-
StatementType type = parseStatementType(sql);
221+
223222
if (type == StatementType.SELECT || type == StatementType.SHOW || type == StatementType.DESCRIBE || type == StatementType.EXPLAIN) {
224223
throw new SQLException("executeUpdate() cannot be called with a SELECT/SHOW/DESCRIBE/EXPLAIN statement", ExceptionUtils.SQL_STATE_SQL_ERROR);
225224
}
@@ -344,18 +343,16 @@ public void setCursorName(String name) throws SQLException {
344343
@Override
345344
public boolean execute(String sql) throws SQLException {
346345
checkClosed();
347-
return execute(sql, new QuerySettings().setDatabase(schema));
346+
return executeImpl(sql, parseStatementType(sql), new QuerySettings().setDatabase(schema));
348347
}
349348

350-
public boolean execute(String sql, QuerySettings settings) throws SQLException {
349+
public boolean executeImpl(String sql, StatementType type, QuerySettings settings) throws SQLException {
351350
checkClosed();
352-
StatementType type = parseStatementType(sql);
353-
354351
if (type == StatementType.SELECT || type == StatementType.SHOW || type == StatementType.DESCRIBE || type == StatementType.EXPLAIN) {
355-
executeQuery(sql, settings); // keep open to allow getResultSet()
352+
executeQueryImpl(sql, settings); // keep open to allow getResultSet()
356353
return true;
357354
} else if(type == StatementType.SET) {
358-
executeUpdate(sql, settings);
355+
executeUpdateImpl(sql, type, settings);
359356
//SET ROLE
360357
List<String> tokens = JdbcUtils.tokenizeSQL(sql);
361358
if (JdbcUtils.containsIgnoresCase(tokens, "ROLE")) {
@@ -392,15 +389,15 @@ public boolean execute(String sql, QuerySettings settings) throws SQLException {
392389
}
393390
return false;
394391
} else if (type == StatementType.USE) {
395-
executeUpdate(sql, settings);
392+
executeUpdateImpl(sql, type, settings);
396393
//USE Database
397394
List<String> tokens = JdbcUtils.tokenizeSQL(sql);
398395
this.schema = tokens.get(1).replace("\"", "");
399396
connection.setSchema(schema);
400397
LOG.debug("Changed statement schema to {}", schema);
401398
return false;
402399
} else {
403-
executeUpdate(sql, settings);
400+
executeUpdateImpl(sql, type, settings);
404401
return false;
405402
}
406403
}

jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/ExceptionUtils.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,10 @@ public final class ExceptionUtils {
2424
public static final String SQL_STATE_INVALID_SCHEMA = "3F000";
2525
public static final String SQL_STATE_INVALID_TX_STATE = "25000";
2626
public static final String SQL_STATE_DATA_EXCEPTION = "22000";
27+
// Used only when feature is not supported
2728
public static final String SQL_STATE_FEATURE_NOT_SUPPORTED = "0A000";
29+
// Used only when method is called on wrong object type (for example, PreparedStatement.addBatch(String))
30+
public static final String SQL_STATE_WRONG_OBJECT_TYPE = "42809";
2831

2932
private ExceptionUtils() {}//Private constructor
3033

0 commit comments

Comments
 (0)