Skip to content

Commit f5ecac8

Browse files
committed
add test for new java.time type support
1 parent c37aac1 commit f5ecac8

File tree

7 files changed

+156
-21
lines changed

7 files changed

+156
-21
lines changed

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

+5-5
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/internal/AbstractBinaryFormatReader.java

+33-14
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++) {
@@ -279,7 +274,7 @@ public String getString(String colName) {
279274
ClickHouseDataType dataType = schema.getColumnByName(colName).getDataType();
280275
ZonedDateTime zdt = (ZonedDateTime) value;
281276
if (dataType == ClickHouseDataType.Date) {
282-
return zdt.format(com.clickhouse.client.api.DataTypeUtils.DATE_FORMATTER).toString();
277+
return zdt.format(com.clickhouse.client.api.DataTypeUtils.DATE_FORMATTER);
283278
}
284279
return value.toString();
285280
} else {
@@ -369,11 +364,17 @@ public Instant getInstant(String colName) {
369364
return data.atStartOfDay().toInstant(ZoneOffset.UTC);
370365
case DateTime:
371366
case DateTime64:
372-
LocalDateTime dateTime = readValue(colName);
373-
return dateTime.toInstant(column.getTimeZone().toZoneId().getRules().getOffset(dateTime));
374-
367+
Object colValue = readValue(colName);
368+
if (colValue instanceof LocalDateTime) {
369+
LocalDateTime dateTime = (LocalDateTime) colValue;
370+
return dateTime.toInstant(column.getTimeZone().toZoneId().getRules().getOffset(dateTime));
371+
} else {
372+
ZonedDateTime dateTime = (ZonedDateTime) colValue;
373+
return dateTime.toInstant();
374+
}
375+
default:
376+
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
375377
}
376-
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
377378
}
378379

379380
@Override
@@ -386,9 +387,9 @@ public ZonedDateTime getZonedDateTime(String colName) {
386387
case Date:
387388
case Date32:
388389
return readValue(colName);
390+
default:
391+
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
389392
}
390-
391-
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
392393
}
393394

394395
@Override
@@ -725,6 +726,24 @@ public LocalDateTime getLocalDateTime(int index) {
725726
return (LocalDateTime) value;
726727
}
727728

729+
@Override
730+
public OffsetDateTime getOffsetDateTime(String colName) {
731+
Object value = readValue(colName);
732+
if (value instanceof ZonedDateTime) {
733+
return ((ZonedDateTime) value).toOffsetDateTime();
734+
}
735+
return (OffsetDateTime) value;
736+
}
737+
738+
@Override
739+
public OffsetDateTime getOffsetDateTime(int index) {
740+
Object value = readValue(index);
741+
if (value instanceof ZonedDateTime) {
742+
return ((ZonedDateTime) value).toOffsetDateTime();
743+
}
744+
return (OffsetDateTime) value;
745+
}
746+
728747
@Override
729748
public ClickHouseBitmap getClickHouseBitmap(String colName) {
730749
return readValue(colName);

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

+10
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

+18
Original file line numberDiff line numberDiff line change
@@ -489,6 +489,24 @@ public LocalDateTime getLocalDateTime(int index) {
489489
return (LocalDateTime) value;
490490
}
491491

492+
@Override
493+
public OffsetDateTime getOffsetDateTime(String colName) {
494+
Object value = readValue(colName);
495+
if (value instanceof ZonedDateTime) {
496+
return ((ZonedDateTime) value).toOffsetDateTime();
497+
}
498+
return (OffsetDateTime) value;
499+
}
500+
501+
@Override
502+
public OffsetDateTime getOffsetDateTime(int index) {
503+
Object value = readValue(index);
504+
if (value instanceof ZonedDateTime) {
505+
return ((ZonedDateTime) value).toOffsetDateTime();
506+
}
507+
return (OffsetDateTime) value;
508+
}
509+
492510
@Override
493511
public ClickHouseBitmap getClickHouseBitmap(String colName) {
494512
return readValue(colName);

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

+4
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

+7
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@
4343
import java.time.Instant;
4444
import java.time.ZoneId;
4545
import java.time.ZonedDateTime;
46+
import java.time.temporal.ChronoField;
4647
import java.util.ArrayList;
4748
import java.util.Arrays;
4849
import java.util.Collections;
@@ -207,6 +208,12 @@ public void insertPOJOAndReadBack() throws Exception {
207208
Assert.assertEquals(reader.getDouble("float64"), pojo.getFloat64());
208209
Assert.assertEquals(reader.getString("string"), pojo.getString());
209210
Assert.assertEquals(reader.getString("fixedString"), pojo.getFixedString());
211+
Assert.assertTrue(reader.getZonedDateTime("zonedDateTime").isEqual(pojo.getZonedDateTime().withNano(0)));
212+
Assert.assertTrue(reader.getZonedDateTime("zonedDateTime64").isEqual(pojo.getZonedDateTime64()));
213+
Assert.assertTrue(reader.getOffsetDateTime("offsetDateTime").isEqual(pojo.getOffsetDateTime().withNano(0)));
214+
Assert.assertTrue(reader.getOffsetDateTime("offsetDateTime64").isEqual(pojo.getOffsetDateTime64()));
215+
Assert.assertEquals(reader.getInstant("instant"), pojo.getInstant().with(ChronoField.MICRO_OF_SECOND, 0));
216+
Assert.assertEquals(reader.getInstant("instant64"), pojo.getInstant64());
210217
}
211218
}
212219

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

+79-2
Original file line numberDiff line numberDiff line change
@@ -8,8 +8,7 @@
88
import java.net.Inet4Address;
99
import java.net.Inet6Address;
1010
import java.net.UnknownHostException;
11-
import java.time.LocalDate;
12-
import java.time.LocalDateTime;
11+
import java.time.*;
1312
import java.util.ArrayList;
1413
import java.util.Arrays;
1514
import java.util.HashMap;
@@ -67,6 +66,15 @@ public class SamplePOJO {
6766
private LocalDateTime dateTime;
6867
private LocalDateTime dateTime64;
6968

69+
private ZonedDateTime zonedDateTime;
70+
private ZonedDateTime zonedDateTime64;
71+
72+
private OffsetDateTime offsetDateTime;
73+
private OffsetDateTime offsetDateTime64;
74+
75+
private Instant instant;
76+
private Instant instant64;
77+
7078
private UUID uuid;
7179

7280
private byte enum8;
@@ -140,6 +148,15 @@ public SamplePOJO() {
140148
dateTime = LocalDateTime.now();
141149
dateTime64 = LocalDateTime.now();
142150

151+
zonedDateTime = ZonedDateTime.now();
152+
zonedDateTime64 = ZonedDateTime.now();
153+
154+
offsetDateTime = OffsetDateTime.now();
155+
offsetDateTime64 = OffsetDateTime.now();
156+
157+
instant = Instant.now();
158+
instant64 = Instant.now();
159+
143160
uuid = UUID.randomUUID();
144161

145162
enum8 = (byte) random.nextInt(27);
@@ -474,6 +491,54 @@ public void setDateTime64(LocalDateTime dateTime64) {
474491
this.dateTime64 = dateTime64;
475492
}
476493

494+
public ZonedDateTime getZonedDateTime() {
495+
return zonedDateTime;
496+
}
497+
498+
public void setZonedDateTime(ZonedDateTime zonedDateTime) {
499+
this.zonedDateTime = zonedDateTime;
500+
}
501+
502+
public ZonedDateTime getZonedDateTime64() {
503+
return zonedDateTime64;
504+
}
505+
506+
public void setZonedDateTime64(ZonedDateTime zonedDateTime64) {
507+
this.zonedDateTime64 = zonedDateTime64;
508+
}
509+
510+
public OffsetDateTime getOffsetDateTime() {
511+
return offsetDateTime;
512+
}
513+
514+
public void setOffsetDateTime(OffsetDateTime offsetDateTime) {
515+
this.offsetDateTime = offsetDateTime;
516+
}
517+
518+
public OffsetDateTime getOffsetDateTime64() {
519+
return offsetDateTime64;
520+
}
521+
522+
public void setOffsetDateTime64(OffsetDateTime offsetDateTime64) {
523+
this.offsetDateTime64 = offsetDateTime64;
524+
}
525+
526+
public Instant getInstant() {
527+
return instant;
528+
}
529+
530+
public void setInstant(Instant instant) {
531+
this.instant = instant;
532+
}
533+
534+
public Instant getInstant64() {
535+
return instant64;
536+
}
537+
538+
public void setInstant64(Instant instant64) {
539+
this.instant64 = instant64;
540+
}
541+
477542
public UUID getUuid() {
478543
return uuid;
479544
}
@@ -628,6 +693,12 @@ public String toString() {
628693
", date32=" + date32 +
629694
", dateTime=" + dateTime +
630695
", dateTime64=" + dateTime64 +
696+
", zonedDateTime=" + zonedDateTime +
697+
", zonedDateTime64=" + zonedDateTime64 +
698+
", offsetDateTime=" + offsetDateTime +
699+
", offsetDateTime64=" + offsetDateTime64 +
700+
", instant=" + instant +
701+
", instant64=" + instant64 +
631702
", uuid=" + uuid +
632703
", enum8=" + enum8 +
633704
", enum16=" + enum16 +
@@ -684,6 +755,12 @@ public static String generateTableCreateSQL(String tableName) {
684755
"date32 Date, " +
685756
"dateTime DateTime, " +
686757
"dateTime64 DateTime64(3), " +
758+
"zonedDateTime DateTime, " +
759+
"zonedDateTime64 DateTime64(9), " +
760+
"offsetDateTime DateTime, " +
761+
"offsetDateTime64 DateTime64(9), " +
762+
"instant DateTime, " +
763+
"instant64 DateTime64(9), " +
687764
"uuid UUID, " +
688765
"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), " +
689766
"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), " +

0 commit comments

Comments
 (0)