Skip to content

Commit a5b77cb

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

File tree

3 files changed

+28
-14
lines changed

3 files changed

+28
-14
lines changed

clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseUtils.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -362,7 +362,7 @@ public static List<Path> findFiles(String pattern, String... paths) throws IOExc
362362
pattern.chars().anyMatch(
363363
value -> {
364364
if (value < ' ' || reservedCharsWindows.indexOf(value) != -1) {
365-
throw new IllegalArgumentException("File path contains reserved character <%s>".formatted((char) value));
365+
throw new IllegalArgumentException(String.format("File path contains reserved character <%s>", value));
366366
}
367367
return false;
368368
}

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

+26-13
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,22 @@ 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;
639+
long ts;
643640
if (value instanceof LocalDateTime) {
644641
LocalDateTime dt = (LocalDateTime) value;
645642
ts = dt.atZone(targetTz).toEpochSecond();
646643
} else if (value instanceof ZonedDateTime) {
647644
ZonedDateTime dt = (ZonedDateTime) value;
648-
ts = dt.withZoneSameInstant(targetTz).toEpochSecond();
645+
ts = dt.toEpochSecond();
649646
} else if (value instanceof Timestamp) {
650647
Timestamp t = (Timestamp) value;
651648
ts = t.toLocalDateTime().atZone(targetTz).toEpochSecond();
649+
} else if(value instanceof OffsetDateTime) {
650+
OffsetDateTime dt = (OffsetDateTime) value;
651+
ts = dt.toEpochSecond();
652+
} else if (value instanceof Instant) {
653+
Instant dt = (Instant) value;
654+
ts = dt.getEpochSecond();
652655
} else {
653656
throw new IllegalArgumentException("Cannot convert " + value + " to DataTime");
654657
}
@@ -661,20 +664,30 @@ public static void writeDateTime64(OutputStream output, Object value, int scale,
661664
throw new IllegalArgumentException("Invalid scale value '" + scale + "'");
662665
}
663666

664-
long ts = 0;
665-
long nano = 0;
667+
long ts;
668+
long nano;
666669
if (value instanceof LocalDateTime) {
667-
ZonedDateTime dt = ((LocalDateTime) value).atZone(targetTz);
668-
ts = dt.toEpochSecond();
669-
nano = dt.getNano();
670+
LocalDateTime dt = (LocalDateTime) value;
671+
ZonedDateTime zdt = dt.atZone(targetTz);
672+
ts = zdt.toEpochSecond();
673+
nano = zdt.getNano();
670674
} else if (value instanceof ZonedDateTime) {
671-
ZonedDateTime dt = ((ZonedDateTime) value).withZoneSameInstant(targetTz);
675+
ZonedDateTime dt = (ZonedDateTime) value;
672676
ts = dt.toEpochSecond();
673677
nano = dt.getNano();
674678
} else if (value instanceof Timestamp) {
675-
ZonedDateTime dt = ((Timestamp) value).toLocalDateTime().atZone(targetTz);
679+
Timestamp dt = (Timestamp) value;
680+
ZonedDateTime zdt = dt.toLocalDateTime().atZone(targetTz);
681+
ts = zdt.toEpochSecond();
682+
nano = zdt.getNano();
683+
} else if (value instanceof OffsetDateTime) {
684+
OffsetDateTime dt = (OffsetDateTime) value;
676685
ts = dt.toEpochSecond();
677686
nano = dt.getNano();
687+
} else if (value instanceof Instant) {
688+
Instant dt = (Instant) value;
689+
ts = dt.getEpochSecond();
690+
nano = dt.getNano();
678691
} else {
679692
throw new IllegalArgumentException("Cannot convert " + value + " to DataTime");
680693
}

0 commit comments

Comments
 (0)