Skip to content

Commit d4336a6

Browse files
authored
Merge pull request #2116 from abcfy2/main
SerializerUtils support OffsetDateTime and Instant
2 parents d716d85 + 29f5ffd commit d4336a6

File tree

11 files changed

+204
-39
lines changed

11 files changed

+204
-39
lines changed

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

Lines changed: 1 addition & 1 deletion
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

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2006,6 +2006,7 @@ private <T> CompletableFuture<T> runAsyncOperation(Supplier<T> resultSupplier, M
20062006
return isAsync ? CompletableFuture.supplyAsync(resultSupplier, sharedOperationExecutor) : CompletableFuture.completedFuture(resultSupplier.get());
20072007
}
20082008

2009+
@Override
20092010
public String toString() {
20102011
return "Client{" +
20112012
"endpoints=" + endpoints +

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

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -11,11 +11,7 @@
1111
import java.math.BigInteger;
1212
import java.net.Inet4Address;
1313
import java.net.Inet6Address;
14-
import java.time.Duration;
15-
import java.time.Instant;
16-
import java.time.LocalDate;
17-
import java.time.LocalDateTime;
18-
import java.time.ZonedDateTime;
14+
import java.time.*;
1915
import java.util.List;
2016
import java.util.Map;
2117
import java.util.UUID;
@@ -535,6 +531,10 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable {
535531

536532
LocalDateTime getLocalDateTime(int index);
537533

534+
OffsetDateTime getOffsetDateTime(String colName);
535+
536+
OffsetDateTime getOffsetDateTime(int index);
537+
538538
TableSchema getSchema();
539539

540540
ClickHouseBitmap getClickHouseBitmap(String colName);

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

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,8 @@
1212
import java.math.BigInteger;
1313
import java.net.Inet4Address;
1414
import java.net.Inet6Address;
15+
import java.time.Instant;
16+
import java.time.OffsetDateTime;
1517
import java.time.ZoneId;
1618
import java.time.ZonedDateTime;
1719

@@ -145,6 +147,22 @@ public void writeDateTime64(ZonedDateTime value, int scale, ZoneId targetTz) thr
145147
SerializerUtils.writeDateTime64(out, value, scale, targetTz);
146148
}
147149

150+
public void writeDateTime32(OffsetDateTime value) throws IOException {
151+
SerializerUtils.writeDateTime32(out, value, null);
152+
}
153+
154+
public void writeDateTime64(OffsetDateTime value, int scale) throws IOException {
155+
SerializerUtils.writeDateTime64(out, value, scale, null);
156+
}
157+
158+
public void writeDateTime32(Instant value) throws IOException {
159+
SerializerUtils.writeDateTime32(out, value, null);
160+
}
161+
162+
public void writeDateTime64(Instant value, int scale) throws IOException {
163+
SerializerUtils.writeDateTime64(out, value, scale, null);
164+
}
165+
148166
public void writeEnum8(byte value) throws IOException {
149167
BinaryStreamUtils.writeEnum8(out, value);
150168
}

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

Lines changed: 33 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -27,12 +27,7 @@
2727
import java.math.BigInteger;
2828
import java.net.Inet4Address;
2929
import java.net.Inet6Address;
30-
import java.time.Duration;
31-
import java.time.Instant;
32-
import java.time.LocalDate;
33-
import java.time.LocalDateTime;
34-
import java.time.ZoneOffset;
35-
import java.time.ZonedDateTime;
30+
import java.time.*;
3631
import java.time.format.DateTimeFormatter;
3732
import java.time.temporal.ChronoUnit;
3833
import java.util.*;
@@ -221,7 +216,7 @@ protected void endReached() {
221216

222217
protected void setSchema(TableSchema schema) {
223218
this.schema = schema;
224-
this.columns = schema.getColumns().toArray(new ClickHouseColumn[0]);
219+
this.columns = schema.getColumns().toArray(ClickHouseColumn.EMPTY_ARRAY);
225220
this.convertions = new Map[columns.length];
226221

227222
for (int i = 0; i < columns.length; i++) {
@@ -287,7 +282,7 @@ public static String readAsString(Object value, ClickHouseColumn column) {
287282
ClickHouseDataType dataType = column.getDataType();
288283
ZonedDateTime zdt = (ZonedDateTime) value;
289284
if (dataType == ClickHouseDataType.Date) {
290-
return zdt.format(com.clickhouse.client.api.DataTypeUtils.DATE_FORMATTER).toString();
285+
return zdt.format(com.clickhouse.client.api.DataTypeUtils.DATE_FORMATTER);
291286
}
292287
return value.toString();
293288
} else if (value instanceof Number ) {
@@ -386,11 +381,17 @@ public Instant getInstant(String colName) {
386381
return data.atStartOfDay().toInstant(ZoneOffset.UTC);
387382
case DateTime:
388383
case DateTime64:
389-
LocalDateTime dateTime = readValue(colName);
390-
return dateTime.toInstant(column.getTimeZone().toZoneId().getRules().getOffset(dateTime));
391-
384+
Object colValue = readValue(colName);
385+
if (colValue instanceof LocalDateTime) {
386+
LocalDateTime dateTime = (LocalDateTime) colValue;
387+
return dateTime.toInstant(column.getTimeZone().toZoneId().getRules().getOffset(dateTime));
388+
} else {
389+
ZonedDateTime dateTime = (ZonedDateTime) colValue;
390+
return dateTime.toInstant();
391+
}
392+
default:
393+
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
392394
}
393-
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
394395
}
395396

396397
@Override
@@ -403,9 +404,9 @@ public ZonedDateTime getZonedDateTime(String colName) {
403404
case Date:
404405
case Date32:
405406
return readValue(colName);
407+
default:
408+
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
406409
}
407-
408-
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
409410
}
410411

411412
@Override
@@ -742,6 +743,24 @@ public LocalDateTime getLocalDateTime(int index) {
742743
return (LocalDateTime) value;
743744
}
744745

746+
@Override
747+
public OffsetDateTime getOffsetDateTime(String colName) {
748+
Object value = readValue(colName);
749+
if (value instanceof ZonedDateTime) {
750+
return ((ZonedDateTime) value).toOffsetDateTime();
751+
}
752+
return (OffsetDateTime) value;
753+
}
754+
755+
@Override
756+
public OffsetDateTime getOffsetDateTime(int index) {
757+
Object value = readValue(index);
758+
if (value instanceof ZonedDateTime) {
759+
return ((ZonedDateTime) value).toOffsetDateTime();
760+
}
761+
return (OffsetDateTime) value;
762+
}
763+
745764
@Override
746765
public ClickHouseBitmap getClickHouseBitmap(String colName) {
747766
return readValue(colName);

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

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -354,6 +354,16 @@ public LocalDateTime getLocalDateTime(int index) {
354354
return reader.getLocalDateTime(index);
355355
}
356356

357+
@Override
358+
public OffsetDateTime getOffsetDateTime(String colName) {
359+
return reader.getOffsetDateTime(colName);
360+
}
361+
362+
@Override
363+
public OffsetDateTime getOffsetDateTime(int index) {
364+
return reader.getOffsetDateTime(index);
365+
}
366+
357367
@Override
358368
public Object getObject(String colName) {
359369
return reader.readValue(colName);

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

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -476,6 +476,24 @@ public LocalDateTime getLocalDateTime(int index) {
476476
return (LocalDateTime) value;
477477
}
478478

479+
@Override
480+
public OffsetDateTime getOffsetDateTime(String colName) {
481+
Object value = readValue(colName);
482+
if (value instanceof ZonedDateTime) {
483+
return ((ZonedDateTime) value).toOffsetDateTime();
484+
}
485+
return (OffsetDateTime) value;
486+
}
487+
488+
@Override
489+
public OffsetDateTime getOffsetDateTime(int index) {
490+
Object value = readValue(index);
491+
if (value instanceof ZonedDateTime) {
492+
return ((ZonedDateTime) value).toOffsetDateTime();
493+
}
494+
return (OffsetDateTime) value;
495+
}
496+
479497
@Override
480498
public ClickHouseBitmap getClickHouseBitmap(String colName) {
481499
return readValue(colName);

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

Lines changed: 26 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -30,10 +30,7 @@
3030
import java.net.Inet4Address;
3131
import java.net.Inet6Address;
3232
import java.sql.Timestamp;
33-
import java.time.LocalDate;
34-
import java.time.LocalDateTime;
35-
import java.time.ZoneId;
36-
import java.time.ZonedDateTime;
33+
import java.time.*;
3734
import java.util.Arrays;
3835
import java.util.HashSet;
3936
import java.util.List;
@@ -716,16 +713,22 @@ public static void writeDateTime32(OutputStream output, Object value, ZoneId tar
716713
}
717714

718715
public static void writeDateTime(OutputStream output, Object value, ZoneId targetTz) throws IOException {
719-
long ts = 0;
716+
long ts;
720717
if (value instanceof LocalDateTime) {
721718
LocalDateTime dt = (LocalDateTime) value;
722719
ts = dt.atZone(targetTz).toEpochSecond();
723720
} else if (value instanceof ZonedDateTime) {
724721
ZonedDateTime dt = (ZonedDateTime) value;
725-
ts = dt.withZoneSameInstant(targetTz).toEpochSecond();
722+
ts = dt.toEpochSecond();
726723
} else if (value instanceof Timestamp) {
727724
Timestamp t = (Timestamp) value;
728725
ts = t.toLocalDateTime().atZone(targetTz).toEpochSecond();
726+
} else if(value instanceof OffsetDateTime) {
727+
OffsetDateTime dt = (OffsetDateTime) value;
728+
ts = dt.toEpochSecond();
729+
} else if (value instanceof Instant) {
730+
Instant dt = (Instant) value;
731+
ts = dt.getEpochSecond();
729732
} else {
730733
throw new IllegalArgumentException("Cannot convert " + value + " to DataTime");
731734
}
@@ -738,20 +741,30 @@ public static void writeDateTime64(OutputStream output, Object value, int scale,
738741
throw new IllegalArgumentException("Invalid scale value '" + scale + "'");
739742
}
740743

741-
long ts = 0;
742-
long nano = 0;
744+
long ts;
745+
long nano;
743746
if (value instanceof LocalDateTime) {
744-
ZonedDateTime dt = ((LocalDateTime) value).atZone(targetTz);
745-
ts = dt.toEpochSecond();
746-
nano = dt.getNano();
747+
LocalDateTime dt = (LocalDateTime) value;
748+
ZonedDateTime zdt = dt.atZone(targetTz);
749+
ts = zdt.toEpochSecond();
750+
nano = zdt.getNano();
747751
} else if (value instanceof ZonedDateTime) {
748-
ZonedDateTime dt = ((ZonedDateTime) value).withZoneSameInstant(targetTz);
752+
ZonedDateTime dt = (ZonedDateTime) value;
749753
ts = dt.toEpochSecond();
750754
nano = dt.getNano();
751755
} else if (value instanceof Timestamp) {
752-
ZonedDateTime dt = ((Timestamp) value).toLocalDateTime().atZone(targetTz);
756+
Timestamp dt = (Timestamp) value;
757+
ZonedDateTime zdt = dt.toLocalDateTime().atZone(targetTz);
758+
ts = zdt.toEpochSecond();
759+
nano = zdt.getNano();
760+
} else if (value instanceof OffsetDateTime) {
761+
OffsetDateTime dt = (OffsetDateTime) value;
753762
ts = dt.toEpochSecond();
754763
nano = dt.getNano();
764+
} else if (value instanceof Instant) {
765+
Instant dt = (Instant) value;
766+
ts = dt.getEpochSecond();
767+
nano = dt.getNano();
755768
} else {
756769
throw new IllegalArgumentException("Cannot convert " + value + " to DataTime");
757770
}

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

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -487,6 +487,10 @@ public interface GenericRecord {
487487

488488
LocalDateTime getLocalDateTime(int index);
489489

490+
OffsetDateTime getOffsetDateTime(String colName);
491+
492+
OffsetDateTime getOffsetDateTime(int index);
493+
490494
Object getObject(String colName);
491495

492496
Object getObject(int index);

client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java

Lines changed: 9 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -48,6 +48,7 @@
4848
import java.time.Instant;
4949
import java.time.ZoneId;
5050
import java.time.ZonedDateTime;
51+
import java.time.temporal.ChronoField;
5152
import java.util.ArrayList;
5253
import java.util.Arrays;
5354
import java.util.Collections;
@@ -139,7 +140,6 @@ public void insertSimplePOJOs() throws Exception {
139140
assertEquals(response.getQueryId(), uuid);
140141
}
141142

142-
143143
@Test(groups = { "integration" }, enabled = true)
144144
public void insertPOJOWithJSON() throws Exception {
145145
if (isCloud()) {
@@ -212,6 +212,12 @@ public void insertPOJOAndReadBack() throws Exception {
212212
Assert.assertEquals(reader.getDouble("float64"), pojo.getFloat64());
213213
Assert.assertEquals(reader.getString("string"), pojo.getString());
214214
Assert.assertEquals(reader.getString("fixedString"), pojo.getFixedString());
215+
Assert.assertTrue(reader.getZonedDateTime("zonedDateTime").isEqual(pojo.getZonedDateTime().withNano(0)));
216+
Assert.assertTrue(reader.getZonedDateTime("zonedDateTime64").isEqual(pojo.getZonedDateTime64()));
217+
Assert.assertTrue(reader.getOffsetDateTime("offsetDateTime").isEqual(pojo.getOffsetDateTime().withNano(0)));
218+
Assert.assertTrue(reader.getOffsetDateTime("offsetDateTime64").isEqual(pojo.getOffsetDateTime64()));
219+
Assert.assertEquals(reader.getInstant("instant"), pojo.getInstant().with(ChronoField.MICRO_OF_SECOND, 0));
220+
Assert.assertEquals(reader.getInstant("instant64"), pojo.getInstant64());
215221
}
216222
}
217223

@@ -456,12 +462,12 @@ public void testWriter() throws Exception {
456462
if (row[4] == null) {
457463
formatWriter.writeDefault();
458464
} else {
459-
formatWriter.writeString((String) row[4]);
465+
formatWriter.writeDateTime((ZonedDateTime) row[4], null);
460466
}
461467
if (row[5] == null) {
462468
formatWriter.writeDefault();
463469
} else {
464-
formatWriter.writeInt8((byte) row[5]);
470+
formatWriter.writeInt8(((Integer) row[5]).byteValue());
465471
}
466472
}
467473
}, ClickHouseFormat.RowBinaryWithDefaults, new InsertSettings()).get()) {

0 commit comments

Comments
 (0)