diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseUtils.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseUtils.java index d90bef810..d3f08d4a8 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseUtils.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseUtils.java @@ -362,7 +362,7 @@ public static List findFiles(String pattern, String... paths) throws IOExc pattern.chars().anyMatch( value -> { if (value < ' ' || reservedCharsWindows.indexOf(value) != -1) { - throw new IllegalArgumentException("File path contains reserved character <%s>".formatted((char) value)); + throw new IllegalArgumentException(String.format("File path contains reserved character <%s>", value)); } return false; } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java index 5b37a8f88..b8c7561f5 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java @@ -2000,6 +2000,7 @@ private CompletableFuture runAsyncOperation(Supplier resultSupplier, M return isAsync ? CompletableFuture.supplyAsync(resultSupplier, sharedOperationExecutor) : CompletableFuture.completedFuture(resultSupplier.get()); } + @Override public String toString() { return "Client{" + "endpoints=" + endpoints + diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java index b361f34d0..b0edfac56 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java @@ -11,11 +11,7 @@ import java.math.BigInteger; import java.net.Inet4Address; import java.net.Inet6Address; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.ZonedDateTime; +import java.time.*; import java.util.List; import java.util.Map; import java.util.UUID; @@ -535,6 +531,10 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable { LocalDateTime getLocalDateTime(int index); + OffsetDateTime getOffsetDateTime(String colName); + + OffsetDateTime getOffsetDateTime(int index); + TableSchema getSchema(); ClickHouseBitmap getClickHouseBitmap(String colName); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryFormatSerializer.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryFormatSerializer.java index 53ff372ae..9dc045053 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryFormatSerializer.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryFormatSerializer.java @@ -12,6 +12,8 @@ import java.math.BigInteger; import java.net.Inet4Address; import java.net.Inet6Address; +import java.time.Instant; +import java.time.OffsetDateTime; import java.time.ZoneId; import java.time.ZonedDateTime; @@ -145,6 +147,22 @@ public void writeDateTime64(ZonedDateTime value, int scale, ZoneId targetTz) thr SerializerUtils.writeDateTime64(out, value, scale, targetTz); } + public void writeDateTime32(OffsetDateTime value) throws IOException { + SerializerUtils.writeDateTime32(out, value, null); + } + + public void writeDateTime64(OffsetDateTime value, int scale) throws IOException { + SerializerUtils.writeDateTime64(out, value, scale, null); + } + + public void writeDateTime32(Instant value) throws IOException { + SerializerUtils.writeDateTime32(out, value, null); + } + + public void writeDateTime64(Instant value, int scale) throws IOException { + SerializerUtils.writeDateTime64(out, value, scale, null); + } + public void writeEnum8(byte value) throws IOException { BinaryStreamUtils.writeEnum8(out, value); } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java index cab0afa6f..a3582575f 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java @@ -27,12 +27,7 @@ import java.math.BigInteger; import java.net.Inet4Address; import java.net.Inet6Address; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.ZoneOffset; -import java.time.ZonedDateTime; +import java.time.*; import java.time.format.DateTimeFormatter; import java.time.temporal.ChronoUnit; import java.util.*; @@ -221,7 +216,7 @@ protected void endReached() { protected void setSchema(TableSchema schema) { this.schema = schema; - this.columns = schema.getColumns().toArray(new ClickHouseColumn[0]); + this.columns = schema.getColumns().toArray(ClickHouseColumn.EMPTY_ARRAY); this.convertions = new Map[columns.length]; for (int i = 0; i < columns.length; i++) { @@ -279,7 +274,7 @@ public String getString(String colName) { ClickHouseDataType dataType = schema.getColumnByName(colName).getDataType(); ZonedDateTime zdt = (ZonedDateTime) value; if (dataType == ClickHouseDataType.Date) { - return zdt.format(com.clickhouse.client.api.DataTypeUtils.DATE_FORMATTER).toString(); + return zdt.format(com.clickhouse.client.api.DataTypeUtils.DATE_FORMATTER); } return value.toString(); } else { @@ -369,11 +364,17 @@ public Instant getInstant(String colName) { return data.atStartOfDay().toInstant(ZoneOffset.UTC); case DateTime: case DateTime64: - LocalDateTime dateTime = readValue(colName); - return dateTime.toInstant(column.getTimeZone().toZoneId().getRules().getOffset(dateTime)); - + Object colValue = readValue(colName); + if (colValue instanceof LocalDateTime) { + LocalDateTime dateTime = (LocalDateTime) colValue; + return dateTime.toInstant(column.getTimeZone().toZoneId().getRules().getOffset(dateTime)); + } else { + ZonedDateTime dateTime = (ZonedDateTime) colValue; + return dateTime.toInstant(); + } + default: + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant"); } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant"); } @Override @@ -386,9 +387,9 @@ public ZonedDateTime getZonedDateTime(String colName) { case Date: case Date32: return readValue(colName); + default: + throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant"); } - - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant"); } @Override @@ -725,6 +726,24 @@ public LocalDateTime getLocalDateTime(int index) { return (LocalDateTime) value; } + @Override + public OffsetDateTime getOffsetDateTime(String colName) { + Object value = readValue(colName); + if (value instanceof ZonedDateTime) { + return ((ZonedDateTime) value).toOffsetDateTime(); + } + return (OffsetDateTime) value; + } + + @Override + public OffsetDateTime getOffsetDateTime(int index) { + Object value = readValue(index); + if (value instanceof ZonedDateTime) { + return ((ZonedDateTime) value).toOffsetDateTime(); + } + return (OffsetDateTime) value; + } + @Override public ClickHouseBitmap getClickHouseBitmap(String colName) { return readValue(colName); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java index 6ed9cb52b..abf26015e 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java @@ -354,6 +354,16 @@ public LocalDateTime getLocalDateTime(int index) { return reader.getLocalDateTime(index); } + @Override + public OffsetDateTime getOffsetDateTime(String colName) { + return reader.getOffsetDateTime(colName); + } + + @Override + public OffsetDateTime getOffsetDateTime(int index) { + return reader.getOffsetDateTime(index); + } + @Override public Object getObject(String colName) { return reader.readValue(colName); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java index 6c514c4a6..d68267e4e 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java @@ -489,6 +489,24 @@ public LocalDateTime getLocalDateTime(int index) { return (LocalDateTime) value; } + @Override + public OffsetDateTime getOffsetDateTime(String colName) { + Object value = readValue(colName); + if (value instanceof ZonedDateTime) { + return ((ZonedDateTime) value).toOffsetDateTime(); + } + return (OffsetDateTime) value; + } + + @Override + public OffsetDateTime getOffsetDateTime(int index) { + Object value = readValue(index); + if (value instanceof ZonedDateTime) { + return ((ZonedDateTime) value).toOffsetDateTime(); + } + return (OffsetDateTime) value; + } + @Override public ClickHouseBitmap getClickHouseBitmap(String colName) { return readValue(colName); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/SerializerUtils.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/SerializerUtils.java index fa4015192..2bba99971 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/SerializerUtils.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/SerializerUtils.java @@ -23,10 +23,7 @@ import java.net.Inet4Address; import java.net.Inet6Address; import java.sql.Timestamp; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.time.ZonedDateTime; +import java.time.*; import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; @@ -639,16 +636,22 @@ public static void writeDateTime32(OutputStream output, Object value, ZoneId tar } public static void writeDateTime(OutputStream output, Object value, ZoneId targetTz) throws IOException { - long ts = 0; + long ts; if (value instanceof LocalDateTime) { LocalDateTime dt = (LocalDateTime) value; ts = dt.atZone(targetTz).toEpochSecond(); } else if (value instanceof ZonedDateTime) { ZonedDateTime dt = (ZonedDateTime) value; - ts = dt.withZoneSameInstant(targetTz).toEpochSecond(); + ts = dt.toEpochSecond(); } else if (value instanceof Timestamp) { Timestamp t = (Timestamp) value; ts = t.toLocalDateTime().atZone(targetTz).toEpochSecond(); + } else if(value instanceof OffsetDateTime) { + OffsetDateTime dt = (OffsetDateTime) value; + ts = dt.toEpochSecond(); + } else if (value instanceof Instant) { + Instant dt = (Instant) value; + ts = dt.getEpochSecond(); } else { throw new IllegalArgumentException("Cannot convert " + value + " to DataTime"); } @@ -661,20 +664,30 @@ public static void writeDateTime64(OutputStream output, Object value, int scale, throw new IllegalArgumentException("Invalid scale value '" + scale + "'"); } - long ts = 0; - long nano = 0; + long ts; + long nano; if (value instanceof LocalDateTime) { - ZonedDateTime dt = ((LocalDateTime) value).atZone(targetTz); - ts = dt.toEpochSecond(); - nano = dt.getNano(); + LocalDateTime dt = (LocalDateTime) value; + ZonedDateTime zdt = dt.atZone(targetTz); + ts = zdt.toEpochSecond(); + nano = zdt.getNano(); } else if (value instanceof ZonedDateTime) { - ZonedDateTime dt = ((ZonedDateTime) value).withZoneSameInstant(targetTz); + ZonedDateTime dt = (ZonedDateTime) value; ts = dt.toEpochSecond(); nano = dt.getNano(); } else if (value instanceof Timestamp) { - ZonedDateTime dt = ((Timestamp) value).toLocalDateTime().atZone(targetTz); + Timestamp dt = (Timestamp) value; + ZonedDateTime zdt = dt.toLocalDateTime().atZone(targetTz); + ts = zdt.toEpochSecond(); + nano = zdt.getNano(); + } else if (value instanceof OffsetDateTime) { + OffsetDateTime dt = (OffsetDateTime) value; ts = dt.toEpochSecond(); nano = dt.getNano(); + } else if (value instanceof Instant) { + Instant dt = (Instant) value; + ts = dt.getEpochSecond(); + nano = dt.getNano(); } else { throw new IllegalArgumentException("Cannot convert " + value + " to DataTime"); } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java index 56f08b3c5..db27d43e1 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java @@ -487,6 +487,10 @@ public interface GenericRecord { LocalDateTime getLocalDateTime(int index); + OffsetDateTime getOffsetDateTime(String colName); + + OffsetDateTime getOffsetDateTime(int index); + Object getObject(String colName); Object getObject(int index); diff --git a/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java b/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java index a7fb4e43e..1920b424b 100644 --- a/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java @@ -43,6 +43,7 @@ import java.time.Instant; import java.time.ZoneId; import java.time.ZonedDateTime; +import java.time.temporal.ChronoField; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -134,7 +135,6 @@ public void insertSimplePOJOs() throws Exception { assertEquals(response.getQueryId(), uuid); } - @Test(groups = { "integration" }, enabled = true) public void insertPOJOWithJSON() throws Exception { if (isCloud()) { @@ -207,6 +207,12 @@ public void insertPOJOAndReadBack() throws Exception { Assert.assertEquals(reader.getDouble("float64"), pojo.getFloat64()); Assert.assertEquals(reader.getString("string"), pojo.getString()); Assert.assertEquals(reader.getString("fixedString"), pojo.getFixedString()); + Assert.assertTrue(reader.getZonedDateTime("zonedDateTime").isEqual(pojo.getZonedDateTime().withNano(0))); + Assert.assertTrue(reader.getZonedDateTime("zonedDateTime64").isEqual(pojo.getZonedDateTime64())); + Assert.assertTrue(reader.getOffsetDateTime("offsetDateTime").isEqual(pojo.getOffsetDateTime().withNano(0))); + Assert.assertTrue(reader.getOffsetDateTime("offsetDateTime64").isEqual(pojo.getOffsetDateTime64())); + Assert.assertEquals(reader.getInstant("instant"), pojo.getInstant().with(ChronoField.MICRO_OF_SECOND, 0)); + Assert.assertEquals(reader.getInstant("instant64"), pojo.getInstant64()); } } @@ -451,12 +457,12 @@ public void testWriter() throws Exception { if (row[4] == null) { formatWriter.writeDefault(); } else { - formatWriter.writeString((String) row[4]); + formatWriter.writeDateTime((ZonedDateTime) row[4], null); } if (row[5] == null) { formatWriter.writeDefault(); } else { - formatWriter.writeInt8((byte) row[5]); + formatWriter.writeInt8(((Integer) row[5]).byteValue()); } } }, ClickHouseFormat.RowBinaryWithDefaults, new InsertSettings()).get()) { diff --git a/client-v2/src/test/java/com/clickhouse/client/insert/SamplePOJO.java b/client-v2/src/test/java/com/clickhouse/client/insert/SamplePOJO.java index 9a55e823b..abc6aebd1 100644 --- a/client-v2/src/test/java/com/clickhouse/client/insert/SamplePOJO.java +++ b/client-v2/src/test/java/com/clickhouse/client/insert/SamplePOJO.java @@ -8,8 +8,7 @@ import java.net.Inet4Address; import java.net.Inet6Address; import java.net.UnknownHostException; -import java.time.LocalDate; -import java.time.LocalDateTime; +import java.time.*; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -67,6 +66,15 @@ public class SamplePOJO { private LocalDateTime dateTime; private LocalDateTime dateTime64; + private ZonedDateTime zonedDateTime; + private ZonedDateTime zonedDateTime64; + + private OffsetDateTime offsetDateTime; + private OffsetDateTime offsetDateTime64; + + private Instant instant; + private Instant instant64; + private UUID uuid; private byte enum8; @@ -115,7 +123,6 @@ public SamplePOJO() { uint32 = (long) (random.nextDouble() * 4294967295L); uint64 = (long) (random.nextDouble() * 18446744073709615L); - uint128 = upper.or(lower).abs(); uint256 = upper1.or(upper2).or(lower1).or(lower2).abs(); @@ -140,6 +147,15 @@ public SamplePOJO() { dateTime = LocalDateTime.now(); dateTime64 = LocalDateTime.now(); + zonedDateTime = ZonedDateTime.now(); + zonedDateTime64 = ZonedDateTime.now(); + + offsetDateTime = OffsetDateTime.now(); + offsetDateTime64 = OffsetDateTime.now(); + + instant = Instant.now(); + instant64 = Instant.now(); + uuid = UUID.randomUUID(); enum8 = (byte) random.nextInt(27); @@ -474,6 +490,54 @@ public void setDateTime64(LocalDateTime dateTime64) { this.dateTime64 = dateTime64; } + public ZonedDateTime getZonedDateTime() { + return zonedDateTime; + } + + public void setZonedDateTime(ZonedDateTime zonedDateTime) { + this.zonedDateTime = zonedDateTime; + } + + public ZonedDateTime getZonedDateTime64() { + return zonedDateTime64; + } + + public void setZonedDateTime64(ZonedDateTime zonedDateTime64) { + this.zonedDateTime64 = zonedDateTime64; + } + + public OffsetDateTime getOffsetDateTime() { + return offsetDateTime; + } + + public void setOffsetDateTime(OffsetDateTime offsetDateTime) { + this.offsetDateTime = offsetDateTime; + } + + public OffsetDateTime getOffsetDateTime64() { + return offsetDateTime64; + } + + public void setOffsetDateTime64(OffsetDateTime offsetDateTime64) { + this.offsetDateTime64 = offsetDateTime64; + } + + public Instant getInstant() { + return instant; + } + + public void setInstant(Instant instant) { + this.instant = instant; + } + + public Instant getInstant64() { + return instant64; + } + + public void setInstant64(Instant instant64) { + this.instant64 = instant64; + } + public UUID getUuid() { return uuid; } @@ -628,6 +692,12 @@ public String toString() { ", date32=" + date32 + ", dateTime=" + dateTime + ", dateTime64=" + dateTime64 + + ", zonedDateTime=" + zonedDateTime + + ", zonedDateTime64=" + zonedDateTime64 + + ", offsetDateTime=" + offsetDateTime + + ", offsetDateTime64=" + offsetDateTime64 + + ", instant=" + instant + + ", instant64=" + instant64 + ", uuid=" + uuid + ", enum8=" + enum8 + ", enum16=" + enum16 + @@ -684,6 +754,12 @@ public static String generateTableCreateSQL(String tableName) { "date32 Date, " + "dateTime DateTime, " + "dateTime64 DateTime64(3), " + + "zonedDateTime DateTime, " + + "zonedDateTime64 DateTime64(9), " + + "offsetDateTime DateTime, " + + "offsetDateTime64 DateTime64(9), " + + "instant DateTime, " + + "instant64 DateTime64(9), " + "uuid UUID, " + "enum8 Enum8('a' = 1, 'b' = 2, 'c' = 3, 'd' = 4, 'e' = 5, 'f' = 6, 'g' = 7, 'h' = 8, 'i' = 9, 'j' = 10, 'k' = 11, 'l' = 12, 'm' = 13, 'n' = 14, 'o' = 15, 'p' = 16, 'q' = 17, 'r' = 18, 's' = 19, 't' = 20, 'u' = 21, 'v' = 22, 'w' = 23, 'x' = 24, 'y' = 25, 'z' = 26), " + "enum16 Enum16('a' = 1, 'b' = 2, 'c' = 3, 'd' = 4, 'e' = 5, 'f' = 6, 'g' = 7, 'h' = 8, 'i' = 9, 'j' = 10, 'k' = 11, 'l' = 12, 'm' = 13, 'n' = 14, 'o' = 15, 'p' = 16, 'q' = 17, 'r' = 18, 's' = 19, 't' = 20, 'u' = 21, 'v' = 22, 'w' = 23, 'x' = 24, 'y' = 25, 'z' = 26), " +