Skip to content

Commit 09bc662

Browse files
committed
SerializerUtils support OffsetDateTime and Instant
1 parent fdf6430 commit 09bc662

File tree

2 files changed

+25
-21
lines changed

2 files changed

+25
-21
lines changed

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

+1
Original file line numberDiff line numberDiff line change
@@ -1994,6 +1994,7 @@ private <T> CompletableFuture<T> runAsyncOperation(Supplier<T> resultSupplier, M
19941994
return isAsync ? CompletableFuture.supplyAsync(resultSupplier, sharedOperationExecutor) : CompletableFuture.completedFuture(resultSupplier.get());
19951995
}
19961996

1997+
@Override
19971998
public String toString() {
19981999
return "Client{" +
19992000
"endpoints=" + endpoints +

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

+24-21
Original file line numberDiff line numberDiff line change
@@ -23,10 +23,7 @@
2323
import java.net.Inet4Address;
2424
import java.net.Inet6Address;
2525
import java.sql.Timestamp;
26-
import java.time.LocalDate;
27-
import java.time.LocalDateTime;
28-
import java.time.ZoneId;
29-
import java.time.ZonedDateTime;
26+
import java.time.*;
3027
import java.util.Arrays;
3128
import java.util.HashMap;
3229
import java.util.HashSet;
@@ -639,16 +636,17 @@ public static void writeDateTime32(OutputStream output, Object value, ZoneId tar
639636
}
640637

641638
public static void writeDateTime(OutputStream output, Object value, ZoneId targetTz) throws IOException {
642-
long ts = 0;
643-
if (value instanceof LocalDateTime) {
644-
LocalDateTime dt = (LocalDateTime) value;
639+
long ts;
640+
if (value instanceof LocalDateTime dt) {
645641
ts = dt.atZone(targetTz).toEpochSecond();
646-
} else if (value instanceof ZonedDateTime) {
647-
ZonedDateTime dt = (ZonedDateTime) value;
648-
ts = dt.withZoneSameInstant(targetTz).toEpochSecond();
649-
} else if (value instanceof Timestamp) {
650-
Timestamp t = (Timestamp) value;
642+
} else if (value instanceof ZonedDateTime dt) {
643+
ts = dt.toEpochSecond();
644+
} else if (value instanceof Timestamp t) {
651645
ts = t.toLocalDateTime().atZone(targetTz).toEpochSecond();
646+
} else if(value instanceof OffsetDateTime dt) {
647+
ts = dt.toEpochSecond();
648+
} else if (value instanceof Instant dt) {
649+
ts = dt.getEpochSecond();
652650
} else {
653651
throw new IllegalArgumentException("Cannot convert " + value + " to DataTime");
654652
}
@@ -661,19 +659,24 @@ public static void writeDateTime64(OutputStream output, Object value, int scale,
661659
throw new IllegalArgumentException("Invalid scale value '" + scale + "'");
662660
}
663661

664-
long ts = 0;
665-
long nano = 0;
666-
if (value instanceof LocalDateTime) {
667-
ZonedDateTime dt = ((LocalDateTime) value).atZone(targetTz);
662+
long ts;
663+
long nano;
664+
if (value instanceof LocalDateTime dt) {
665+
ZonedDateTime zdt = dt.atZone(targetTz);
666+
ts = zdt.toEpochSecond();
667+
nano = zdt.getNano();
668+
} else if (value instanceof ZonedDateTime dt) {
668669
ts = dt.toEpochSecond();
669670
nano = dt.getNano();
670-
} else if (value instanceof ZonedDateTime) {
671-
ZonedDateTime dt = ((ZonedDateTime) value).withZoneSameInstant(targetTz);
671+
} else if (value instanceof Timestamp dt) {
672+
ZonedDateTime zdt = dt.toLocalDateTime().atZone(targetTz);
673+
ts = zdt.toEpochSecond();
674+
nano = zdt.getNano();
675+
} else if (value instanceof OffsetDateTime dt) {
672676
ts = dt.toEpochSecond();
673677
nano = dt.getNano();
674-
} else if (value instanceof Timestamp) {
675-
ZonedDateTime dt = ((Timestamp) value).toLocalDateTime().atZone(targetTz);
676-
ts = dt.toEpochSecond();
678+
} else if (value instanceof Instant dt) {
679+
ts = dt.getEpochSecond();
677680
nano = dt.getNano();
678681
} else {
679682
throw new IllegalArgumentException("Cannot convert " + value + " to DataTime");

0 commit comments

Comments
 (0)