Skip to content

Commit fae09e1

Browse files
committed
implemented handling timezones in readers
1 parent e1eb107 commit fae09e1

File tree

5 files changed

+148
-31
lines changed

5 files changed

+148
-31
lines changed

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

Lines changed: 43 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -531,16 +531,41 @@ public Builder setClientKey(String path) {
531531
return this;
532532
}
533533

534-
public Builder setUseServerTimeZone(boolean useServerTimeZone) {
534+
/**
535+
* Configure client to use server timezone for date/datetime columns. Default is true.
536+
* If this options is selected then server timezone should be set as well.
537+
*
538+
* @param useServerTimeZone - if to use server timezone
539+
* @return
540+
*/
541+
public Builder useServerTimeZone(boolean useServerTimeZone) {
535542
this.configuration.put(ClickHouseClientOption.USE_SERVER_TIME_ZONE.getKey(), String.valueOf(useServerTimeZone));
536543
return this;
537544
}
538545

539-
public Builder setUseTimeZone(String timeZone) {
546+
/**
547+
* Configure client to use specified timezone. If set using server TimeZone should be
548+
* set to false
549+
*
550+
* @param timeZone
551+
* @return
552+
*/
553+
public Builder useTimeZone(String timeZone) {
540554
this.configuration.put(ClickHouseClientOption.USE_TIME_ZONE.getKey(), timeZone);
541555
return this;
542556
}
543557

558+
/**
559+
* Specify server timezone to use. If not set then UTC will be used.
560+
*
561+
* @param timeZone - server timezone
562+
* @return
563+
*/
564+
public Builder setServerTimeZone(String timeZone) {
565+
this.configuration.put(ClickHouseClientOption.SERVER_TIME_ZONE.getKey(), timeZone);
566+
return this;
567+
}
568+
544569
public Client build() {
545570
this.configuration = setDefaults(this.configuration);
546571

@@ -560,19 +585,31 @@ public Client build() {
560585

561586
// Check timezone settings
562587
String useTimeZoneValue = this.configuration.get(ClickHouseClientOption.USE_TIME_ZONE.getKey());
588+
String serverTimeZoneValue = this.configuration.get(ClickHouseClientOption.SERVER_TIME_ZONE.getKey());
589+
boolean useServerTimeZone = MapUtils.getFlag(this.configuration, ClickHouseClientOption.USE_SERVER_TIME_ZONE.getKey());
563590
if (useTimeZoneValue != null) {
564-
if (MapUtils.getFlag(this.configuration,
565-
ClickHouseClientOption.USE_SERVER_TIME_ZONE.getKey())) {
591+
if (useServerTimeZone) {
566592
throw new IllegalArgumentException("USE_TIME_ZONE option cannot be used when using server timezone");
567593
}
568594

569595
try {
570-
LOG.info("Using timezone: {} instead of service one", ZoneId.of(useTimeZoneValue));
596+
LOG.info("Using timezone: {} instead of server one", ZoneId.of(useTimeZoneValue));
571597
} catch (Exception e) {
572598
throw new IllegalArgumentException("Invalid timezone value: " + useTimeZoneValue);
573599
}
574-
}
600+
} else if (useServerTimeZone) {
601+
if (serverTimeZoneValue == null) {
602+
serverTimeZoneValue = "UTC";
603+
}
575604

605+
try {
606+
LOG.info("Using server timezone: {}", ZoneId.of(serverTimeZoneValue));
607+
} catch (Exception e) {
608+
throw new IllegalArgumentException("Invalid server timezone value: " + serverTimeZoneValue);
609+
}
610+
} else {
611+
throw new IllegalArgumentException("Nor server timezone nor specific timezone is set");
612+
}
576613

577614
return new Client(this.endpoints, this.configuration, this.useNewImplementation);
578615
}

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

Lines changed: 11 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -566,8 +566,8 @@ public short getEnum16(int index) {
566566
@Override
567567
public LocalDate getLocalDate(String colName) {
568568
Object value = readValue(colName);
569-
if (value instanceof LocalDateTime) {
570-
return ((LocalDateTime) value).toLocalDate();
569+
if (value instanceof ZonedDateTime) {
570+
return ((ZonedDateTime) value).toLocalDate();
571571
}
572572
return (LocalDate) value;
573573

@@ -576,23 +576,27 @@ public LocalDate getLocalDate(String colName) {
576576
@Override
577577
public LocalDate getLocalDate(int index) {
578578
Object value = readValue(index);
579-
if (value instanceof LocalDateTime) {
580-
return ((LocalDateTime) value).toLocalDate();
579+
if (value instanceof ZonedDateTime) {
580+
return ((ZonedDateTime) value).toLocalDate();
581581
}
582582
return (LocalDate) value;
583583
}
584584

585585
@Override
586586
public LocalDateTime getLocalDateTime(String colName) {
587587
Object value = readValue(colName);
588-
if (value instanceof LocalDate) {
589-
return ((LocalDate) value).atStartOfDay();
588+
if (value instanceof ZonedDateTime) {
589+
return ((ZonedDateTime) value).toLocalDateTime();
590590
}
591591
return (LocalDateTime) value;
592592
}
593593

594594
@Override
595595
public LocalDateTime getLocalDateTime(int index) {
596-
return readValue(index);
596+
Object value = readValue(index);
597+
if (value instanceof ZonedDateTime) {
598+
return ((ZonedDateTime) value).toLocalDateTime();
599+
}
600+
return (LocalDateTime) value;
597601
}
598602
}

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

Lines changed: 10 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
import java.time.Instant;
2020
import java.time.LocalDate;
2121
import java.time.LocalDateTime;
22+
import java.time.ZonedDateTime;
2223
import java.util.Collections;
2324
import java.util.LinkedHashMap;
2425
import java.util.Map;
@@ -450,34 +451,25 @@ public static BigInteger readUnsignedInt256LE(InputStream input) throws IOExcept
450451
return new BigInteger(1, readNBytes(input, 32));
451452
}
452453

453-
public static LocalDate readDate(InputStream input, TimeZone tz)
454-
throws IOException {
454+
public static ZonedDateTime readDate(InputStream input, TimeZone tz) throws IOException {
455455
LocalDate d = LocalDate.ofEpochDay(readUnsignedShortLE(input));
456-
if (tz != null && !tz.toZoneId().equals(ClickHouseValues.SYS_ZONE)) {
457-
d = d.atStartOfDay(ClickHouseValues.SYS_ZONE).withZoneSameInstant(tz.toZoneId()).toLocalDate();
458-
}
459-
return d;
456+
return d.atStartOfDay(tz.toZoneId()).withZoneSameInstant(tz.toZoneId());
460457
}
461458

462-
public static LocalDate readDate32(InputStream input, TimeZone tz)
459+
public static ZonedDateTime readDate32(InputStream input, TimeZone tz)
463460
throws IOException {
464461
LocalDate d = LocalDate.ofEpochDay(readIntLE(input));
465-
if (tz != null && !tz.toZoneId().equals(ClickHouseValues.SYS_ZONE)) {
466-
d = d.atStartOfDay(ClickHouseValues.SYS_ZONE).withZoneSameInstant(tz.toZoneId()).toLocalDate();
467-
}
468-
return d;
462+
return d.atStartOfDay(tz.toZoneId()).withZoneSameInstant(tz.toZoneId());
469463
}
470464

471-
public static LocalDateTime readDateTime32(InputStream input, TimeZone tz) throws IOException {
465+
public static ZonedDateTime readDateTime32(InputStream input, TimeZone tz) throws IOException {
472466
long time = readUnsignedIntLE(input);
473-
474-
return LocalDateTime.ofInstant(Instant.ofEpochSecond(Math.max(time, 0L)),
475-
tz != null ? tz.toZoneId() : ClickHouseValues.UTC_ZONE);
467+
return LocalDateTime.ofInstant(Instant.ofEpochSecond(Math.max(time, 0L)), tz.toZoneId()).atZone(tz.toZoneId());
476468
}
477469
private static final int[] BASES = new int[] { 1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000,
478470
1000000000 };
479471

480-
public static LocalDateTime readDateTime64(InputStream input, int scale, TimeZone tz) throws IOException {
472+
public static ZonedDateTime readDateTime64(InputStream input, int scale, TimeZone tz) throws IOException {
481473
long value = readLongLE(input);
482474
int nanoSeconds = 0;
483475
if (scale > 0) {
@@ -493,8 +485,8 @@ public static LocalDateTime readDateTime64(InputStream input, int scale, TimeZon
493485
}
494486
}
495487

496-
return LocalDateTime.ofInstant(Instant.ofEpochSecond(value, nanoSeconds),
497-
tz != null ? tz.toZoneId() : TimeZone.getTimeZone("UTC").toZoneId());
488+
return LocalDateTime.ofInstant(Instant.ofEpochSecond(value, nanoSeconds), tz.toZoneId())
489+
.atZone(tz.toZoneId());
498490
}
499491

500492
public static String readString(InputStream input) throws IOException {

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

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -190,4 +190,8 @@ public TimeZone getTimeZone() {
190190
? null
191191
: (TimeZone) settings.getOption("server_timezone");
192192
}
193+
194+
public QuerySettings getSettings() {
195+
return settings;
196+
}
193197
}

client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java

Lines changed: 80 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -52,6 +52,9 @@
5252
import java.math.BigInteger;
5353
import java.time.LocalDate;
5454
import java.time.LocalDateTime;
55+
import java.time.OffsetDateTime;
56+
import java.time.ZoneId;
57+
import java.time.ZonedDateTime;
5558
import java.time.temporal.ChronoUnit;
5659
import java.util.ArrayList;
5760
import java.util.Arrays;
@@ -1166,4 +1169,81 @@ public void testGetTableSchemaError() {
11661169

11671170
}
11681171
}
1172+
1173+
@Test(groups = {"integration"})
1174+
public void testServerTimeZoneFromHeader() {
1175+
1176+
final String requestTimeZone = "America/Los_Angeles";
1177+
try (QueryResponse response =
1178+
client.query("SELECT now() as t, toDateTime(now(), 'UTC') as utc_time " +
1179+
"SETTINGS session_timezone = '" + requestTimeZone + "'").get(1, TimeUnit.SECONDS)) {
1180+
1181+
ClickHouseBinaryFormatReader reader =
1182+
new RowBinaryWithNamesAndTypesFormatReader(response.getInputStream(), response.getSettings());
1183+
1184+
reader.next();
1185+
1186+
LocalDateTime serverTime = reader.getLocalDateTime(1);
1187+
System.out.println("Server time: " + serverTime);
1188+
LocalDateTime serverUtcTime = reader.getLocalDateTime(2);
1189+
System.out.println("Server UTC time: " + serverUtcTime);
1190+
1191+
ZonedDateTime serverTimeZ = serverTime.atZone(ZoneId.of(requestTimeZone));
1192+
ZonedDateTime serverUtcTimeZ = serverUtcTime.atZone(ZoneId.of("UTC"));
1193+
1194+
Assert.assertEquals(serverTimeZ.withZoneSameInstant(ZoneId.of("UTC")), serverUtcTimeZ);
1195+
1196+
} catch (Exception e) {
1197+
e.printStackTrace();
1198+
Assert.fail("Failed to get server time zone from header", e);
1199+
}
1200+
}
1201+
1202+
1203+
@Test(groups = {"integration"})
1204+
public void testClientUseOwnTimeZone() {
1205+
1206+
final String overrideTz = "America/Los_Angeles";
1207+
try (Client client = newClient().useTimeZone(overrideTz).useServerTimeZone(false).build()) {
1208+
final String requestTimeZone = "Europe/Berlin";
1209+
try (QueryResponse response =
1210+
client.query("SELECT now() as t, toDateTime(now(), 'UTC') as utc_time, " +
1211+
"toDateTime(now(), 'Europe/Lisbon')" +
1212+
"SETTINGS session_timezone = '" + requestTimeZone + "'").get(1, TimeUnit.SECONDS)) {
1213+
1214+
ClickHouseBinaryFormatReader reader =
1215+
new RowBinaryWithNamesAndTypesFormatReader(response.getInputStream(), response.getSettings());
1216+
1217+
reader.next();
1218+
1219+
LocalDateTime serverTime = reader.getLocalDateTime(1); // in "America/Los_Angeles"
1220+
System.out.println("Server time: " + serverTime);
1221+
LocalDateTime serverUtcTime = reader.getLocalDateTime(2);
1222+
System.out.println("Server UTC time: " + serverUtcTime);
1223+
ZonedDateTime serverLisbonTime = reader.getZonedDateTime(3); // in "Europe/Lisbon"
1224+
System.out.println("Server Lisbon time: " + serverLisbonTime);
1225+
1226+
ZonedDateTime serverTimeZ = serverTime.atZone(ZoneId.of("America/Los_Angeles"));
1227+
ZonedDateTime serverUtcTimeZ = serverUtcTime.atZone(ZoneId.of("UTC"));
1228+
1229+
Assert.assertEquals(serverTimeZ.withZoneSameInstant(ZoneId.of("UTC")), serverUtcTimeZ);
1230+
Assert.assertEquals(serverLisbonTime.withZoneSameInstant(ZoneId.of("UTC")), serverTimeZ);
1231+
}
1232+
} catch (Exception e) {
1233+
e.printStackTrace();
1234+
Assert.fail("Failed to get server time zone from header", e);
1235+
}
1236+
}
1237+
1238+
private Client.Builder newClient() {
1239+
ClickHouseNode node = getServer(ClickHouseProtocol.HTTP);
1240+
return new Client.Builder()
1241+
.addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), false)
1242+
.setUsername("default")
1243+
.setPassword("")
1244+
.compressClientRequest(false)
1245+
.compressServerResponse(false)
1246+
.useNewImplementation(System.getProperty("client.tests.useNewImplementation", "true").equals("true"))
1247+
;
1248+
}
11691249
}

0 commit comments

Comments
 (0)