Skip to content

Commit e1eb107

Browse files
committed
propagation of timezone settings to request settings
1 parent 8649357 commit e1eb107

File tree

6 files changed

+35
-27
lines changed

6 files changed

+35
-27
lines changed

clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseSimpleResponse.java

Lines changed: 9 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -1,12 +1,5 @@
11
package com.clickhouse.client;
22

3-
import java.sql.Time;
4-
import java.util.ArrayList;
5-
import java.util.Collections;
6-
import java.util.LinkedList;
7-
import java.util.List;
8-
import java.util.TimeZone;
9-
103
import com.clickhouse.data.ClickHouseColumn;
114
import com.clickhouse.data.ClickHouseInputStream;
125
import com.clickhouse.data.ClickHouseRecord;
@@ -15,6 +8,12 @@
158
import com.clickhouse.data.ClickHouseSimpleRecord;
169
import com.clickhouse.data.ClickHouseValue;
1710

11+
import java.util.ArrayList;
12+
import java.util.Collections;
13+
import java.util.LinkedList;
14+
import java.util.List;
15+
import java.util.TimeZone;
16+
1817
/**
1918
* A simple response built on top of two lists: columns and records.
2019
*/
@@ -31,7 +30,7 @@ public class ClickHouseSimpleResponse implements ClickHouseResponse {
3130
* @return response object
3231
*/
3332
public static ClickHouseResponse of(ClickHouseConfig config, List<ClickHouseColumn> columns, Object[][] values) {
34-
return of(config, columns, values, null);
33+
return of(config, columns, values, null, null);
3534
}
3635

3736
/**
@@ -136,6 +135,7 @@ protected ClickHouseSimpleResponse(List<ClickHouseColumn> columns, List<ClickHou
136135
this.columns = columns;
137136
this.records = Collections.unmodifiableList(records);
138137
this.summary = summary != null ? summary : ClickHouseResponseSummary.EMPTY;
138+
this.timeZone = timeZone;
139139
}
140140

141141
protected ClickHouseSimpleResponse(List<ClickHouseColumn> columns, ClickHouseValue[][] values,
@@ -151,6 +151,7 @@ protected ClickHouseSimpleResponse(List<ClickHouseColumn> columns, ClickHouseVal
151151
this.records = Collections.unmodifiableList(list);
152152

153153
this.summary = summary != null ? summary : ClickHouseResponseSummary.EMPTY;
154+
this.timeZone = timeZone;
154155
}
155156

156157
@Override

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

Lines changed: 17 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -1051,17 +1051,7 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
10511051
clientStats.start(ClientMetrics.OP_DURATION);
10521052

10531053
if (useNewImplementation) {
1054-
// merge settings
1055-
if (!settings.getAllSettings().containsKey(ClickHouseClientOption.USE_TIME_ZONE.getKey()) &&
1056-
configuration.containsKey(ClickHouseClientOption.USE_TIME_ZONE.getKey())) {
1057-
settings.setOption(ClickHouseClientOption.USE_TIME_ZONE.getKey(),
1058-
configuration.get(ClickHouseClientOption.USE_TIME_ZONE.getKey()));
1059-
}
1060-
if (!settings.getAllSettings().containsKey(ClickHouseClientOption.USE_SERVER_TIME_ZONE.getKey()) &&
1061-
configuration.containsKey(ClickHouseClientOption.USE_SERVER_TIME_ZONE.getKey())) {
1062-
settings.setOption(ClickHouseClientOption.USE_SERVER_TIME_ZONE.getKey(),
1063-
MapUtils.getFlag(configuration, ClickHouseClientOption.USE_SERVER_TIME_ZONE.getKey()));
1064-
}
1054+
applyDefaults(settings);
10651055
//
10661056
String retry = configuration.get(ClickHouseClientOption.RETRY.getKey());
10671057
final int maxRetries = retry == null ? (int) ClickHouseClientOption.RETRY.getDefaultValue() : Integer.parseInt(retry);
@@ -1194,7 +1184,8 @@ public List<GenericRecord> queryAll(String sqlQuery) {
11941184
query(sqlQuery, settings).get(operationTimeout, TimeUnit.MILLISECONDS)) {
11951185
List<GenericRecord> records = new ArrayList<>();
11961186
if (response.getResultRows() > 0) {
1197-
ClickHouseBinaryFormatReader reader = new RowBinaryWithNamesAndTypesFormatReader(response.getInputStream());
1187+
ClickHouseBinaryFormatReader reader =
1188+
new RowBinaryWithNamesAndTypesFormatReader(response.getInputStream(), settings);
11981189
Map<String, Object> record;
11991190
while ((record = reader.next()) != null) {
12001191
records.add(new MapBackedRecord(record, reader.getSchema()));
@@ -1292,6 +1283,20 @@ private String startOperation() {
12921283
return operationId;
12931284
}
12941285

1286+
private void applyDefaults(QuerySettings settings) {
1287+
Map<String, Object> settingsMap = settings.getAllSettings();
1288+
1289+
String key = ClickHouseClientOption.USE_SERVER_TIME_ZONE.getKey();
1290+
if (!settingsMap.containsKey(key) && configuration.containsKey(key)) {
1291+
settings.setOption(key, MapUtils.getFlag(configuration, key));
1292+
}
1293+
1294+
key = ClickHouseClientOption.USE_TIME_ZONE.getKey();
1295+
if ( !settings.getUseServerTimeZone() && !settingsMap.containsKey(key) && configuration.containsKey(key)) {
1296+
settings.setOption(key, TimeZone.getTimeZone(configuration.get(key)));
1297+
}
1298+
}
1299+
12951300
public String toString() {
12961301
return "Client{" +
12971302
"endpoints=" + endpoints +

client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryWithNamesAndTypesFormatReader.java

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

1818
public class RowBinaryWithNamesAndTypesFormatReader extends AbstractBinaryFormatReader implements Iterator<Map<String, Object>> {
1919

20-
public RowBinaryWithNamesAndTypesFormatReader(InputStream inputStream) {
21-
this(inputStream, null);
22-
}
23-
2420
public RowBinaryWithNamesAndTypesFormatReader(InputStream inputStream, QuerySettings querySettings) {
2521
super(inputStream, querySettings, null);
2622
readSchema();

client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -61,7 +61,7 @@ protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings quer
6161
TimeZone timeZone = useServerTimeZone ? querySettings.getServerTimeZone() :
6262
(TimeZone) this.settings.get(ClickHouseClientOption.USE_TIME_ZONE.getKey());
6363
if (timeZone == null) {
64-
throw new ClientException("Time zone is not set");
64+
throw new ClientException("Time zone is not set. (useServerTimezone:" + useServerTimeZone + ")");
6565
}
6666
this.binaryStreamReader = new BinaryStreamReader(inputStream, timeZone, LOG);
6767
setSchema(schema);

client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -64,7 +64,13 @@ public QueryResponse(ClassicHttpResponse response, QuerySettings settings, Opera
6464
this.operationMetrics = operationMetrics;
6565

6666
Header tzHeader = response.getFirstHeader(ClickHouseHttpProto.HEADER_TIMEZONE);
67-
settings.setOption("server_timezone", tzHeader);
67+
if (tzHeader != null) {
68+
try {
69+
settings.setOption("server_timezone", TimeZone.getTimeZone(tzHeader.getValue()));
70+
} catch (Exception e) {
71+
throw new ClientException("Failed to parse server timezone", e);
72+
}
73+
}
6874
}
6975

7076
public InputStream getInputStream() {

client-v2/src/main/java/com/clickhouse/client/api/query/Records.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,7 @@ public Records(QueryResponse response, QuerySettings finalSettings) {
2828
throw new ClientException("Unsupported format: " + finalSettings.getFormat());
2929
}
3030

31-
this.reader = new RowBinaryWithNamesAndTypesFormatReader(response.getInputStream());
31+
this.reader = new RowBinaryWithNamesAndTypesFormatReader(response.getInputStream(), finalSettings);
3232
this.empty = !reader.hasNext();
3333
}
3434

0 commit comments

Comments
 (0)