Skip to content

Commit fa9f5a1

Browse files
committed
Merge branch 'main' into jdbc_fix_nested_types
2 parents 0aaaf96 + d475e82 commit fa9f5a1

File tree

18 files changed

+1590
-501
lines changed

18 files changed

+1590
-501
lines changed

clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseServerForTest.java

Lines changed: 0 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -370,13 +370,6 @@ public static String getDatabase() {
370370

371371
public static boolean runQuery(String sql) {
372372
LOGGER.info("runQuery: (\"" + sql + "\")");
373-
374-
try {
375-
throw new Exception("test");
376-
} catch (Exception e) {
377-
e.printStackTrace();
378-
}
379-
380373
if (clickhouseContainer != null) {
381374
try {
382375
Container.ExecResult res = clickhouseContainer.execInContainer("clickhouse-client",

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

Lines changed: 49 additions & 57 deletions
Original file line numberDiff line numberDiff line change
@@ -118,7 +118,7 @@ public class Client implements AutoCloseable {
118118
private HttpAPIClientHelper httpClientHelper = null;
119119

120120
private final List<Endpoint> endpoints;
121-
private final Map<String, String> configuration;
121+
private final Map<String, Object> configuration;
122122

123123
private final Map<String, String> readOnlyConfig;
124124

@@ -150,15 +150,16 @@ private Client(Set<String> endpoints, Map<String,String> configuration,
150150
private Client(Set<String> endpoints, Map<String,String> configuration,
151151
ExecutorService sharedOperationExecutor, ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy, Object metricsRegistry) {
152152
// Simple initialization
153-
this.configuration = configuration;
154-
this.readOnlyConfig = Collections.unmodifiableMap(this.configuration);
153+
this.configuration = ClientConfigProperties.parseConfigMap(configuration);
154+
this.readOnlyConfig = Collections.unmodifiableMap(configuration);
155155
this.metricsRegistry = metricsRegistry;
156156

157157
// Serialization
158158
this.pojoSerDe = new POJOSerDe(columnToMethodMatchingStrategy);
159159

160160
// Operation Execution
161-
boolean isAsyncEnabled = MapUtils.getFlag(this.configuration, ClientConfigProperties.ASYNC_OPERATIONS.getKey(), false);
161+
boolean isAsyncEnabled = ClientConfigProperties.ASYNC_OPERATIONS.getOrDefault(this.configuration);
162+
162163
if (isAsyncEnabled && sharedOperationExecutor == null) {
163164
this.isSharedOpExecutorOwned = true;
164165
this.sharedOperationExecutor = Executors.newCachedThreadPool(new DefaultThreadFactory("chc-operation"));
@@ -184,7 +185,7 @@ private Client(Set<String> endpoints, Map<String,String> configuration,
184185
}
185186

186187
this.endpoints = tmpEndpoints.build();
187-
this.httpClientHelper = new HttpAPIClientHelper(configuration, metricsRegistry, initSslContext);
188+
this.httpClientHelper = new HttpAPIClientHelper(this.configuration, metricsRegistry, initSslContext);
188189

189190
String retry = configuration.get(ClientConfigProperties.RETRY_ON_FAILURE.getKey());
190191
this.retries = retry == null ? 0 : Integer.parseInt(retry);
@@ -242,7 +243,7 @@ public void loadServerInfo() {
242243
* @return String - actual default database name.
243244
*/
244245
public String getDefaultDatabase() {
245-
return this.configuration.get("database");
246+
return (String) this.configuration.get(ClientConfigProperties.DATABASE.getKey());
246247
}
247248

248249

@@ -870,7 +871,7 @@ public Builder setMaxRetries(int maxRetries) {
870871
* @return
871872
*/
872873
public Builder allowBinaryReaderToReuseBuffers(boolean reuse) {
873-
this.configuration.put("client_allow_binary_reader_to_reuse_buffers", String.valueOf(reuse));
874+
this.configuration.put(ClientConfigProperties.BINARY_READER_USE_PREALLOCATED_BUFFERS.getKey(), String.valueOf(reuse));
874875
return this;
875876
}
876877

@@ -1048,20 +1049,21 @@ public Client build() {
10481049
throw new IllegalArgumentException("At least one endpoint is required");
10491050
}
10501051
// check if username and password are empty. so can not initiate client?
1051-
if (!this.configuration.containsKey("access_token") &&
1052-
(!this.configuration.containsKey("user") || !this.configuration.containsKey("password")) &&
1053-
!MapUtils.getFlag(this.configuration, "ssl_authentication", false) &&
1054-
!this.configuration.containsKey(ClientConfigProperties.httpHeader(HttpHeaders.AUTHORIZATION))) {
1052+
boolean useSslAuth = MapUtils.getFlag(this.configuration, ClientConfigProperties.SSL_AUTH.getKey());
1053+
boolean hasAccessToken = this.configuration.containsKey(ClientConfigProperties.ACCESS_TOKEN.getKey());
1054+
boolean hasUser = this.configuration.containsKey(ClientConfigProperties.USER.getKey());
1055+
boolean hasPassword = this.configuration.containsKey(ClientConfigProperties.PASSWORD.getKey());
1056+
boolean customHttpHeaders = this.configuration.containsKey(ClientConfigProperties.httpHeader(HttpHeaders.AUTHORIZATION));
1057+
1058+
if (!(useSslAuth || hasAccessToken || hasUser || hasPassword || customHttpHeaders)) {
10551059
throw new IllegalArgumentException("Username and password (or access token or SSL authentication or pre-define Authorization header) are required");
10561060
}
10571061

1058-
if (this.configuration.containsKey("ssl_authentication") &&
1059-
(this.configuration.containsKey("password") || this.configuration.containsKey("access_token"))) {
1062+
if (useSslAuth && (hasAccessToken || hasPassword)) {
10601063
throw new IllegalArgumentException("Only one of password, access token or SSL authentication can be used per client.");
10611064
}
10621065

1063-
if (this.configuration.containsKey("ssl_authentication") &&
1064-
!this.configuration.containsKey(ClientConfigProperties.SSL_CERTIFICATE.getKey())) {
1066+
if (useSslAuth && !this.configuration.containsKey(ClientConfigProperties.SSL_CERTIFICATE.getKey())) {
10651067
throw new IllegalArgumentException("SSL authentication requires a client certificate");
10661068
}
10671069

@@ -1198,17 +1200,16 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
11981200
if (data == null || data.isEmpty()) {
11991201
throw new IllegalArgumentException("Data cannot be empty");
12001202
}
1201-
1203+
//Add format to the settings
1204+
if (settings == null) {
1205+
settings = new InsertSettings();
1206+
}
12021207

12031208
String operationId = registerOperationMetrics();
12041209
settings.setOperationId(operationId);
12051210
globalClientStats.get(operationId).start(ClientMetrics.OP_DURATION);
12061211
globalClientStats.get(operationId).start(ClientMetrics.OP_SERIALIZATION);
12071212

1208-
//Add format to the settings
1209-
if (settings == null) {
1210-
settings = new InsertSettings();
1211-
}
12121213

12131214
boolean hasDefaults = this.tableSchemaHasDefaults.get(tableName);
12141215
ClickHouseFormat format = hasDefaults? ClickHouseFormat.RowBinaryWithDefaults : ClickHouseFormat.RowBinary;
@@ -1232,11 +1233,11 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
12321233
}
12331234

12341235

1235-
String retry = configuration.get(ClientConfigProperties.RETRY_ON_FAILURE.getKey());
1236-
final int maxRetries = retry == null ? 0 : Integer.parseInt(retry);
1236+
Integer retry = (Integer) configuration.get(ClientConfigProperties.RETRY_ON_FAILURE.getKey());
1237+
final int maxRetries = retry == null ? 0 : retry;
12371238

12381239
settings.setOption(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey(), format.name());
1239-
final InsertSettings finalSettings = settings;
1240+
final InsertSettings finalSettings = new InsertSettings(buildRequestSettings(settings.getAllSettings()));
12401241
Supplier<InsertResponse> supplier = () -> {
12411242
long startTime = System.nanoTime();
12421243
// Selecting some node
@@ -1358,8 +1359,7 @@ public CompletableFuture<InsertResponse> insert(String tableName,
13581359
InsertSettings settings) {
13591360

13601361
final int writeBufferSize = settings.getInputStreamCopyBufferSize() <= 0 ?
1361-
Integer.parseInt(configuration.getOrDefault(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey(),
1362-
ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getDefaultValue())) :
1362+
(int) configuration.get(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey()) :
13631363
settings.getInputStreamCopyBufferSize();
13641364

13651365
if (writeBufferSize <= 0) {
@@ -1431,17 +1431,16 @@ public CompletableFuture<InsertResponse> insert(String tableName,
14311431

14321432
Supplier<InsertResponse> responseSupplier;
14331433

1434-
14351434
final int writeBufferSize = settings.getInputStreamCopyBufferSize() <= 0 ?
1436-
Integer.parseInt(configuration.getOrDefault(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey(), "8192")) :
1435+
(int) configuration.get(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey()) :
14371436
settings.getInputStreamCopyBufferSize();
14381437

14391438
if (writeBufferSize <= 0) {
14401439
throw new IllegalArgumentException("Buffer size must be greater than 0");
14411440
}
14421441

14431442
settings.setOption(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey(), format.name());
1444-
final InsertSettings finalSettings = settings;
1443+
final InsertSettings finalSettings = new InsertSettings(buildRequestSettings(settings.getAllSettings()));
14451444

14461445
StringBuilder sqlStmt = new StringBuilder("INSERT INTO ").append(tableName);
14471446
if (columnNames != null && !columnNames.isEmpty()) {
@@ -1571,14 +1570,13 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
15711570
}
15721571
ClientStatisticsHolder clientStats = new ClientStatisticsHolder();
15731572
clientStats.start(ClientMetrics.OP_DURATION);
1574-
applyDefaults(settings);
15751573

15761574
Supplier<QueryResponse> responseSupplier;
15771575

15781576
if (queryParams != null) {
15791577
settings.setOption("statement_params", queryParams);
15801578
}
1581-
final QuerySettings finalSettings = settings;
1579+
final QuerySettings finalSettings = new QuerySettings(buildRequestSettings(settings.getAllSettings()));
15821580
responseSupplier = () -> {
15831581
long startTime = System.nanoTime();
15841582
// Selecting some node
@@ -1956,7 +1954,7 @@ public CompletableFuture<CommandResponse> execute(String sql) {
19561954
public ClickHouseBinaryFormatReader newBinaryFormatReader(QueryResponse response, TableSchema schema) {
19571955
ClickHouseBinaryFormatReader reader = null;
19581956
// Using caching buffer allocator is risky so this parameter is not exposed to the user
1959-
boolean useCachingBufferAllocator = MapUtils.getFlag(configuration, "client_allow_binary_reader_to_reuse_buffers");
1957+
boolean useCachingBufferAllocator = MapUtils.getFlag(configuration, "client_allow_binary_reader_to_reuse_buffers", false);
19601958
BinaryStreamReader.ByteBufferAllocator byteBufferPool = useCachingBufferAllocator ?
19611959
new BinaryStreamReader.CachingByteBufferAllocator() :
19621960
new BinaryStreamReader.DefaultByteBufferAllocator();
@@ -1991,25 +1989,6 @@ private String registerOperationMetrics() {
19911989
return operationId;
19921990
}
19931991

1994-
private void applyDefaults(QuerySettings settings) {
1995-
Map<String, Object> settingsMap = settings.getAllSettings();
1996-
1997-
String key = ClientConfigProperties.USE_SERVER_TIMEZONE.getKey();
1998-
if (!settingsMap.containsKey(key) && configuration.containsKey(key)) {
1999-
settings.setOption(key, MapUtils.getFlag(configuration, key));
2000-
}
2001-
2002-
key = ClientConfigProperties.USE_TIMEZONE.getKey();
2003-
if ( !settings.getUseServerTimeZone() && !settingsMap.containsKey(key) && configuration.containsKey(key)) {
2004-
settings.setOption(key, TimeZone.getTimeZone(configuration.get(key)));
2005-
}
2006-
2007-
key = ClientConfigProperties.SERVER_TIMEZONE.getKey();
2008-
if (!settingsMap.containsKey(key) && configuration.containsKey(key)) {
2009-
settings.setOption(key, TimeZone.getTimeZone(configuration.get(key)));
2010-
}
2011-
}
2012-
20131992
private <T> CompletableFuture<T> runAsyncOperation(Supplier<T> resultSupplier, Map<String, Object> requestSettings) {
20141993
boolean isAsync = MapUtils.getFlag(requestSettings, configuration, ClientConfigProperties.ASYNC_OPERATIONS.getKey());
20151994
if (isAsync) {
@@ -2037,7 +2016,7 @@ public Map<String, String> getConfiguration() {
20372016

20382017
/** Returns operation timeout in seconds */
20392018
protected int getOperationTimeout() {
2040-
return Integer.parseInt(configuration.get(ClientConfigProperties.MAX_EXECUTION_TIME.getKey()));
2019+
return ClientConfigProperties.MAX_EXECUTION_TIME.getOrDefault(configuration);
20412020
}
20422021

20432022
/**
@@ -2050,15 +2029,16 @@ public Set<String> getEndpoints() {
20502029
}
20512030

20522031
public String getUser() {
2053-
return this.configuration.get(ClientConfigProperties.USER.getKey());
2032+
return (String) this.configuration.get(ClientConfigProperties.USER.getKey());
20542033
}
20552034

20562035
public String getServerVersion() {
20572036
return this.serverVersion;
20582037
}
20592038

20602039
public String getServerTimeZone() {
2061-
return this.configuration.get(ClientConfigProperties.SERVER_TIMEZONE.getKey());
2040+
TimeZone tz = (TimeZone) this.configuration.get(ClientConfigProperties.SERVER_TIMEZONE.getKey());
2041+
return tz == null ? null : tz.getID();
20622042
}
20632043

20642044
public String getClientVersion() {
@@ -2071,10 +2051,9 @@ public String getClientVersion() {
20712051
* @param dbRoles
20722052
*/
20732053
public void setDBRoles(Collection<String> dbRoles) {
2074-
this.configuration.put(ClientConfigProperties.SESSION_DB_ROLES.getKey(), ClientConfigProperties.commaSeparated(dbRoles));
2075-
this.unmodifiableDbRolesView =
2076-
Collections.unmodifiableCollection(ClientConfigProperties.valuesFromCommaSeparated(
2077-
this.configuration.get(ClientConfigProperties.SESSION_DB_ROLES.getKey())));
2054+
List<String> tmp = new ArrayList<>(dbRoles);
2055+
this.configuration.put(ClientConfigProperties.SESSION_DB_ROLES.getKey(), tmp);
2056+
this.unmodifiableDbRolesView = ImmutableList.copyOf(tmp);
20782057
}
20792058

20802059
public void updateClientName(String name) {
@@ -2105,4 +2084,17 @@ private Endpoint getNextAliveNode() {
21052084
}
21062085

21072086
public static final String VALUES_LIST_DELIMITER = ",";
2087+
2088+
/**
2089+
* Produces a merge of operation and client settings.
2090+
* Operation settings override client settings
2091+
* @param opSettings - operation settings
2092+
* @return request settings - merged client and operation settings
2093+
*/
2094+
private Map<String, Object> buildRequestSettings(Map<String, Object> opSettings) {
2095+
Map<String, Object> requestSettings = new HashMap<>();
2096+
requestSettings.putAll(configuration);
2097+
requestSettings.putAll(opSettings);
2098+
return requestSettings;
2099+
}
21082100
}

0 commit comments

Comments
 (0)