Skip to content

Commit

Permalink
add test for new java.time type support
Browse files Browse the repository at this point in the history
  • Loading branch information
abcfy2 committed Jan 31, 2025
1 parent c37aac1 commit f5ecac8
Show file tree
Hide file tree
Showing 7 changed files with 156 additions and 21 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.*;
Expand Down Expand Up @@ -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++) {
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -207,6 +208,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());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -140,6 +148,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);
Expand Down Expand Up @@ -474,6 +491,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;
}
Expand Down Expand Up @@ -628,6 +693,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 +
Expand Down Expand Up @@ -684,6 +755,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), " +
Expand Down

0 comments on commit f5ecac8

Please sign in to comment.