diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java index 0e88403f1..2ae64bf99 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java @@ -211,6 +211,7 @@ private static ClickHouseColumn update(ClickHouseColumn column) { } break; case DateTime32: + case Time: if (size > 0) { column.template = ClickHouseOffsetDateTimeValue.ofNull( column.scale, @@ -219,6 +220,7 @@ private static ClickHouseColumn update(ClickHouseColumn column) { } break; case DateTime64: + case Time64: if (size > 0) { column.scale = Integer.parseInt(column.parameters.get(0)); } diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java index 38849e5be..9cba74c82 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -5,6 +5,7 @@ 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.LocalTime; @@ -70,7 +71,7 @@ public enum ClickHouseDataType { // https://docs.oracle.com/javase/tutorial/java/nutsandbolts/datatypes.html#PageTitle UInt32(UnsignedInteger.class, false, true, false, 4, 10, 0, 0, 0, false, 0x03, "INT UNSIGNED", "INTEGER UNSIGNED", "MEDIUMINT UNSIGNED"), - Int64(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x0A,"BIGINT", "BIGINT SIGNED", "TIME"), + Int64(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x0A,"BIGINT", "BIGINT SIGNED"), IntervalYear(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22), IntervalQuarter(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22), IntervalMonth(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22), @@ -127,6 +128,8 @@ public enum ClickHouseDataType { AggregateFunction(String.class, true, true, false, 0, 0, 0, 0, 0, true), Variant(List.class, true, true, false, 0, 0, 0, 0, 0, true, 0x2A), Dynamic(Object.class, true, true, false, 0, 0, 0, 0, 0, true, 0x2B), + Time(LocalDateTime.class, true, false, false, 4, 9, 0, 0, 9, false, 0x32), // 0x33 for Time(Timezone) + Time64(LocalDateTime.class, true, false, false, 8, 9, 0, 0, 0, false, 0x34), // 0x35 for Time64(P, Timezone) ; public static final List ORDERED_BY_RANGE_INT_TYPES = @@ -237,6 +240,9 @@ static Map>> dataTypeClassMap() { map.put(IntervalMicrosecond, timeIntervalClasses); map.put(IntervalNanosecond, timeIntervalClasses); + map.put(Time, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(Integer.class, Long.class, Instant.class)))); + map.put(Time64, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(Integer.class, Long.class, BigInteger.class, Instant.class)))); + return map; } diff --git a/clickhouse-data/src/test/java/com/clickhouse/data/ClickHouseColumnTest.java b/clickhouse-data/src/test/java/com/clickhouse/data/ClickHouseColumnTest.java index f42009e0d..a04f42d32 100644 --- a/clickhouse-data/src/test/java/com/clickhouse/data/ClickHouseColumnTest.java +++ b/clickhouse-data/src/test/java/com/clickhouse/data/ClickHouseColumnTest.java @@ -420,7 +420,8 @@ public boolean isWidenUnsignedTypes() { // skip advanced types if (type.isNested() || type == ClickHouseDataType.AggregateFunction || type == ClickHouseDataType.SimpleAggregateFunction || type == ClickHouseDataType.Enum - || type == ClickHouseDataType.Nullable || type == ClickHouseDataType.BFloat16) { + || type == ClickHouseDataType.Nullable || type == ClickHouseDataType.BFloat16 || + type == ClickHouseDataType.Time || type == ClickHouseDataType.Time64) { continue; } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/DataTypeUtils.java b/client-v2/src/main/java/com/clickhouse/client/api/DataTypeUtils.java index d0a49bc7a..6accd1f0b 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/DataTypeUtils.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/DataTypeUtils.java @@ -1,5 +1,9 @@ package com.clickhouse.client.api; +import java.time.Instant; +import java.time.ZoneId; +import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader; + import java.time.Instant; import java.time.ZoneId; import java.time.format.DateTimeFormatter; @@ -9,6 +13,8 @@ import com.clickhouse.data.ClickHouseDataType; +import static com.clickhouse.client.api.data_formats.internal.BinaryStreamReader.BASES; + public class DataTypeUtils { /** @@ -113,4 +119,21 @@ private static String formatInstantDefault(Instant instant) { return INSTANT_FORMATTER.format(instant); } + public static Instant instantFromTime64Integer(int precision, long value) { + int nanoSeconds = 0; + if (precision > 0) { + int factor = BinaryStreamReader.BASES[precision]; + nanoSeconds = (int) (value % factor); + value /= factor; + if (nanoSeconds < 0) { + nanoSeconds += factor; + value--; + } + if (nanoSeconds > 0L) { + nanoSeconds *= BASES[9 - precision]; + } + } + + return Instant.ofEpochSecond(value, nanoSeconds); + } } 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 ab11b731b..04a0bdb17 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 @@ -2,6 +2,7 @@ import com.clickhouse.client.api.ClientConfigProperties; import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.DataTypeUtils; import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; import com.clickhouse.client.api.internal.MapUtils; import com.clickhouse.client.api.internal.ServerSettings; @@ -304,6 +305,8 @@ protected void setSchema(TableSchema schema) { case Enum16: case Variant: case Dynamic: + case Time: + case Time64: this.convertions[i] = NumberConverter.NUMBER_CONVERTERS; break; default: @@ -452,6 +455,10 @@ public Instant getInstant(String colName) { ZonedDateTime dateTime = (ZonedDateTime) colValue; return dateTime.toInstant(); } + case Time: + return Instant.ofEpochSecond(getLong(colName)); + case Time64: + return DataTypeUtils.instantFromTime64Integer(column.getScale(), getLong(colName)); default: throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant"); } @@ -472,7 +479,7 @@ public ZonedDateTime getZonedDateTime(String colName) { 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 ZonedDateTime"); } } @@ -671,7 +678,7 @@ public BigDecimal getBigDecimal(int index) { @Override public Instant getInstant(int index) { - return readValue(index); + return getInstant(schema.columnIndexToName(index)); } @Override @@ -706,22 +713,22 @@ public UUID getUUID(int index) { @Override public ClickHouseGeoPointValue getGeoPoint(int index) { - return readValue(index); + return getGeoPoint(schema.columnIndexToName(index)); } @Override public ClickHouseGeoRingValue getGeoRing(int index) { - return readValue(index); + return getGeoRing(schema.columnIndexToName(index)); } @Override public ClickHouseGeoPolygonValue getGeoPolygon(int index) { - return readValue(index); + return getGeoPolygon(schema.columnIndexToName(index)); } @Override public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { - return readValue(index); + return getGeoMultiPolygon(schema.columnIndexToName(index)); } @Override @@ -803,11 +810,7 @@ public LocalDate getLocalDate(String colName) { @Override public LocalDate getLocalDate(int index) { - Object value = readValue(index); - if (value instanceof ZonedDateTime) { - return ((ZonedDateTime) value).toLocalDate(); - } - return (LocalDate) value; + return getLocalDate(schema.columnIndexToName(index)); } @Override @@ -821,11 +824,7 @@ public LocalDateTime getLocalDateTime(String colName) { @Override public LocalDateTime getLocalDateTime(int index) { - Object value = readValue(index); - if (value instanceof ZonedDateTime) { - return ((ZonedDateTime) value).toLocalDateTime(); - } - return (LocalDateTime) value; + return getLocalDateTime(schema.columnIndexToName(index)); } @Override @@ -839,11 +838,7 @@ public OffsetDateTime getOffsetDateTime(String colName) { @Override public OffsetDateTime getOffsetDateTime(int index) { - Object value = readValue(index); - if (value instanceof ZonedDateTime) { - return ((ZonedDateTime) value).toOffsetDateTime(); - } - return (OffsetDateTime) value; + return getOffsetDateTime(schema.columnIndexToName(index)); } @Override diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java index 6c55808eb..6ee472456 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java @@ -186,6 +186,10 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce return convertDateTime(readDateTime32(timezone), typeHint); case DateTime64: return convertDateTime(readDateTime64(scale, timezone), typeHint); + case Time: + return (T) (Integer) readIntLE(); + case Time64: + return (T) (Long) (readLongLE()); case IntervalYear: case IntervalQuarter: case IntervalMonth: 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 ef24ae782..23bb5cd07 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 @@ -1,17 +1,28 @@ package com.clickhouse.client.api.data_formats.internal; import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.DataTypeUtils; import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.GenericRecord; import com.clickhouse.client.api.query.NullValueException; import com.clickhouse.data.ClickHouseColumn; -import com.clickhouse.data.value.*; +import com.clickhouse.data.value.ClickHouseBitmap; +import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoPointValue; +import com.clickhouse.data.value.ClickHouseGeoPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoRingValue; import java.math.BigDecimal; import java.math.BigInteger; import java.net.Inet4Address; import java.net.Inet6Address; -import java.time.*; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; import java.time.temporal.TemporalAmount; import java.util.HashMap; import java.util.List; @@ -130,6 +141,11 @@ public Instant getInstant(String colName) { case DateTime64: LocalDateTime dateTime = readValue(colName); return dateTime.toInstant(column.getTimeZone().toZoneId().getRules().getOffset(dateTime)); + case Time: + return Instant.ofEpochSecond(getLong(colName)); + case Time64: + return DataTypeUtils.instantFromTime64Integer(column.getScale(), getLong(colName)); + } throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant"); } @@ -248,7 +264,7 @@ public boolean[] getBooleanArray(String colName) { @Override public boolean hasValue(int colIndex) { - return record.containsKey(schema.columnIndexToName(colIndex)); + return hasValue(schema.columnIndexToName(colIndex)); } @Override @@ -293,27 +309,27 @@ public boolean getBoolean(int index) { @Override public BigInteger getBigInteger(int index) { - return readValue(index); + return getBigInteger(schema.columnIndexToName(index)); } @Override public BigDecimal getBigDecimal(int index) { - return readValue(index); + return getBigDecimal(schema.columnIndexToName(index)); } @Override public Instant getInstant(int index) { - return readValue(index); + return getInstant(schema.columnIndexToName(index)); } @Override public ZonedDateTime getZonedDateTime(int index) { - return readValue(index); + return getZonedDateTime(schema.columnIndexToName(index)); } @Override public Duration getDuration(int index) { - return readValue(index); + return getDuration(schema.columnIndexToName(index)); } @Override @@ -338,22 +354,22 @@ public UUID getUUID(int index) { @Override public ClickHouseGeoPointValue getGeoPoint(int index) { - return readValue(index); + return getGeoPoint(schema.columnIndexToName(index)); } @Override public ClickHouseGeoRingValue getGeoRing(int index) { - return readValue(index); + return getGeoRing(schema.columnIndexToName(index)); } @Override public ClickHouseGeoPolygonValue getGeoPolygon(int index) { - return readValue(index); + return getGeoPolygon(schema.columnIndexToName(index)); } @Override public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { - return readValue(index); + return getGeoMultiPolygon(schema.columnIndexToName(index)); } @Override @@ -423,11 +439,7 @@ public short getEnum16(int index) { @Override public LocalDate getLocalDate(int index) { - Object value = readValue(index); - if (value instanceof ZonedDateTime) { - return ((ZonedDateTime) value).toLocalDate(); - } - return (LocalDate) value; + return getLocalDate(schema.columnIndexToName(index)); } @Override @@ -451,11 +463,7 @@ public LocalDateTime getLocalDateTime(String colName) { @Override public LocalDateTime getLocalDateTime(int index) { - Object value = readValue(index); - if (value instanceof ZonedDateTime) { - return ((ZonedDateTime) value).toLocalDateTime(); - } - return (LocalDateTime) value; + return getLocalDateTime(schema.columnIndexToName(index)); } @Override @@ -469,11 +477,7 @@ public OffsetDateTime getOffsetDateTime(String colName) { @Override public OffsetDateTime getOffsetDateTime(int index) { - Object value = readValue(index); - if (value instanceof ZonedDateTime) { - return ((ZonedDateTime) value).toOffsetDateTime(); - } - return (OffsetDateTime) value; + return getOffsetDateTime(schema.columnIndexToName(index)); } @Override 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 e53a0617f..23dfad81d 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 @@ -200,6 +200,8 @@ public static ClickHouseColumn valueToColumnForDynamicType(Object value) { column = enumValue2Column((Enum)value); } else if (value instanceof List || (value !=null && value.getClass().isArray())) { column = listValue2Column(value); + } else if (value instanceof Instant) { + column = ClickHouseColumn.of("v", "Time64(9)"); } else if (value == null) { column = PREDEFINED_TYPE_COLUMNS.get(Void.class); } else { @@ -400,6 +402,10 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ case AggregateFunction: stream.write(binTag); break; + case Time64: + stream.write(binTag); + BinaryStreamUtils.writeUnsignedInt8(stream, dt.getMaxPrecision()); + break; default: stream.write(binTag); } @@ -547,6 +553,12 @@ private static void serializePrimitiveData(OutputStream stream, Object value, Cl writeDateTime64(stream, value, column.getScale(), zoneId); break; } + case Time: + BinaryStreamUtils.writeInt32(stream, convertToInteger(value)); + break; + case Time64: + serializeTime64(stream, value); + break; case UUID: BinaryStreamUtils.writeUuid(stream, (UUID) value); break; @@ -641,6 +653,19 @@ private static void serializeInterval(OutputStream stream, ClickHouseColumn colu BinaryStreamUtils.writeUnsignedInt64(stream, v); } + private static void serializeTime64(OutputStream stream, Object value) throws IOException { + if (value instanceof BigInteger) { + BinaryStreamUtils.writeUnsignedInt64(stream, (BigInteger) value); + } else if (value instanceof Long) { + BinaryStreamUtils.writeUnsignedInt64(stream, (Long) value); + } else if (value instanceof Instant) { + BinaryStreamUtils.writeUnsignedInt64(stream, BigInteger.valueOf(((Instant) value).getEpochSecond()).shiftLeft(32) + .add(BigInteger.valueOf(((Instant) value).getNano()))); + } else { + throw new UnsupportedOperationException("Cannot convert " + value.getClass() + " to Time64"); + } + } + private static void serializeEnumData(OutputStream stream, ClickHouseColumn column, Object value) throws IOException { int enumValue = -1; if (value instanceof String) { diff --git a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java index 99e822668..b829c63a8 100644 --- a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java @@ -5,11 +5,13 @@ import com.clickhouse.client.ClickHouseProtocol; import com.clickhouse.client.ClickHouseServerForTest; import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.DataTypeUtils; import com.clickhouse.client.api.command.CommandSettings; import com.clickhouse.client.api.enums.Protocol; import com.clickhouse.client.api.insert.InsertSettings; import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.GenericRecord; +import com.clickhouse.client.api.query.QueryResponse; import com.clickhouse.data.ClickHouseDataType; import com.clickhouse.data.ClickHouseVersion; import lombok.AllArgsConstructor; @@ -19,17 +21,19 @@ import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.io.IOException; import java.lang.reflect.Method; import java.math.BigDecimal; -import java.time.Duration; +import java.time.Instant; import java.time.LocalDateTime; import java.time.Period; +import java.time.temporal.ChronoField; import java.time.temporal.ChronoUnit; -import java.time.temporal.TemporalAmount; -import java.time.temporal.TemporalUnit; +import java.time.temporal.TemporalAccessor; +import java.time.temporal.TemporalField; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -38,6 +42,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; @@ -189,6 +194,8 @@ public void testVariantWithSimpleDataTypes() throws Exception { case LowCardinality: // virtual type case LineString: // same as Ring case MultiLineString: // same as MultiPolygon + case Time: + case Time64: // tested separately continue dataTypesLoop; @@ -405,6 +412,32 @@ public void testVariantWithTuple() throws Exception { }); } + @Test(groups = {"integration"}) + public void testVariantWithTime64Types() throws Exception { + if (isVersionMatch("(,25.5]")) { + return; // time64 was introduced in 25.6 + } + testVariantWith("Time", new String[]{"field Variant(Time, String)"}, + new Object[]{ + "30:33:30", + TimeUnit.HOURS.toSeconds(100) + TimeUnit.MINUTES.toSeconds(10) + 30 + }, + new String[]{ + "30:33:30", + "360630", // Time stored as integer by default + }); + + testVariantWith("Time64", new String[]{"field Variant(Time64, String)"}, + new Object[]{ + "30:33:30", + TimeUnit.HOURS.toSeconds(100) + TimeUnit.MINUTES.toSeconds(10) + 30 + }, + new String[]{ + "30:33:30", + "360630", + }); + } + @Test(groups = {"integration"}) public void testDynamicWithPrimitives() throws Exception { if (isVersionMatch("(,24.8]")) { @@ -446,6 +479,8 @@ public void testDynamicWithPrimitives() throws Exception { case Enum: // virtual type case LineString: // same as Ring case MultiLineString: // same as MultiPolygon + case Time: + case Time64: // no tests or tested in other tests continue; default: @@ -599,6 +634,33 @@ public void testDynamicWithMaps() throws Exception { }); } + @Test(groups = {"integration"}) + public void testDynamicWithTime64Types() throws Exception { + if (isVersionMatch("(,25.5]")) { + return; // time64 was introduced in 25.6 + } + + long _999_hours = TimeUnit.HOURS.toSeconds(999); + testDynamicWith("Time", + new Object[]{ + _999_hours + }, + new String[]{ + String.valueOf(_999_hours), + }); + + Instant maxTime64 = Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59, + 999999999); + + testDynamicWith("Time64", + new Object[]{ + maxTime64, + }, + new String[]{ + "3958241016481971977" + }); + } + @Data @AllArgsConstructor public static class DTOForDynamicPrimitivesTests { @@ -629,6 +691,135 @@ public void testAllDataTypesKnown() { Assert.assertTrue(unknowTypes.isEmpty(), "There are some unknown types: " + unknowTypes); } + @Test(groups = {"integration"}) + public void testTimeDataType() throws Exception { + if (isVersionMatch("(,25.5]")) { + return; // time64 was introduced in 25.6 + } + + + String table = "test_time_type"; + client.execute("DROP TABLE IF EXISTS " + table).get(); + client.execute(tableDefinition(table, "o_num UInt32", "time Time"), (CommandSettings) new CommandSettings().serverSetting("allow_experimental_time_time64_type", "1")).get(); + + String insertSQL = "INSERT INTO " + table + " VALUES (1, '999:00:00'), (2, '999:59:59'), (3, '000:00:00'), (4, '-999:59:59')"; + try (QueryResponse response = client.query(insertSQL).get()) {} + + + List records = client.queryAll("SELECT * FROM " + table); + + GenericRecord record = records.get(0); + Assert.assertEquals(record.getInteger("o_num"), 1); + Assert.assertEquals(record.getInteger("time"), TimeUnit.HOURS.toSeconds(999)); + Assert.assertEquals(record.getInstant("time"), Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(999))); + + record = records.get(1); + Assert.assertEquals(record.getInteger("o_num"), 2); + Assert.assertEquals(record.getInteger("time"), TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59); + Assert.assertEquals(record.getInstant("time"), Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59)); + + record = records.get(2); + Assert.assertEquals(record.getInteger("o_num"), 3); + Assert.assertEquals(record.getInteger("time"), 0); + Assert.assertEquals(record.getInstant("time"), Instant.ofEpochSecond(0)); + + record = records.get(3); + Assert.assertEquals(record.getInteger("o_num"), 4); + Assert.assertEquals(record.getInteger("time"), - (TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59)); + Assert.assertEquals(record.getInstant("time"), Instant.ofEpochSecond(- + (TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59))); + } + + @Test(groups = {"integration"}) + public void testTime64() throws Exception { + if (isVersionMatch("(,25.5]")) { + return; // time64 was introduced in 25.6 + } + + String table = "test_time64_type"; + client.execute("DROP TABLE IF EXISTS " + table).get(); + client.execute(tableDefinition(table, "o_num UInt32", "t_sec Time64(0)", "t_ms Time64(3)", "t_us Time64(6)", "t_ns Time64(9)"), + (CommandSettings) new CommandSettings().serverSetting("allow_experimental_time_time64_type", "1")).get(); + + String[][] values = new String[][] { + {"00:01:00.123", "00:01:00.123", "00:01:00.123456", "00:01:00.123456789"}, + {"-00:01:00.123", "-00:01:00.123", "-00:01:00.123456", "-00:01:00.123456789"}, + {"-999:59:59.999", "-999:59:59.999", "-999:59:59.999999", "-999:59:59.999999999"}, + {"999:59:59.999", "999:59:59.999", "999:59:59.999999", "999:59:59.999999999"}, + }; + + Long[][] expectedValues = new Long[][] { + {timeToSec(0, 1,0), timeToMs(0, 1,0) + 123, timeToUs(0, 1,0) + 123456, timeToNs(0, 1,0) + 123456789}, + {-timeToSec(0, 1,0), -(timeToMs(0, 1,0) + 123), -(timeToUs(0, 1,0) + 123456), -(timeToNs(0, 1,0) + 123456789)}, + {-timeToSec(999,59, 59), -(timeToMs(999,59, 59) + 999), + -(timeToUs(999, 59, 59) + 999999), -(timeToNs(999, 59, 59) + 999999999)}, + {timeToSec(999,59, 59), timeToMs(999,59, 59) + 999, + timeToUs(999, 59, 59) + 999999, timeToNs(999, 59, 59) + 999999999}, + }; + + String[][] expectedInstantStrings = new String[][] { + {"1970-01-01T00:01:00Z", + "1970-01-01T00:01:00.123Z", + "1970-01-01T00:01:00.123456Z", + "1970-01-01T00:01:00.123456789Z"}, + + {"1969-12-31T23:59:00Z", + "1969-12-31T23:58:59.877Z", + "1969-12-31T23:58:59.876544Z", + "1969-12-31T23:58:59.876543211Z"}, + + {"1969-11-20T08:00:01Z", + "1969-11-20T08:00:00.001Z", + "1969-11-20T08:00:00.000001Z", + "1969-11-20T08:00:00.000000001Z"}, + + + {"1970-02-11T15:59:59Z", + "1970-02-11T15:59:59.999Z", + "1970-02-11T15:59:59.999999Z", + "1970-02-11T15:59:59.999999999Z"}, + }; + + for (int i = 0; i < values.length; i++) { + StringBuilder insertSQL = new StringBuilder("INSERT INTO " + table + " VALUES (" + i + ", "); + for (int j = 0; j < values[i].length; j++) { + insertSQL.append("'").append(values[i][j]).append("', "); + } + insertSQL.setLength(insertSQL.length() - 2); + insertSQL.append(");"); + + client.query(insertSQL.toString()).get().close(); + + List records = client.queryAll("SELECT * FROM " + table); + + GenericRecord record = records.get(0); + Assert.assertEquals(record.getInteger("o_num"), i); + for (int j = 0; j < values[i].length; j++) { + Assert.assertEquals(record.getLong(j + 2), expectedValues[i][j], "failed at value " +j); + Instant actualInstant = record.getInstant(j + 2); + Assert.assertEquals(actualInstant.toString(), expectedInstantStrings[i][j], "failed at value " +j); + } + + client.execute("TRUNCATE TABLE " + table).get(); + } + } + + private static long timeToSec(int hours, int minutes, int seconds) { + return TimeUnit.HOURS.toSeconds(hours) + TimeUnit.MINUTES.toSeconds(minutes) + seconds; + } + + private static long timeToMs(int hours, int minutes, int seconds) { + return TimeUnit.HOURS.toMillis(hours) + TimeUnit.MINUTES.toMillis(minutes) + TimeUnit.SECONDS.toMillis(seconds); + } + + private static long timeToUs(int hours, int minutes, int seconds) { + return TimeUnit.HOURS.toMicros(hours) + TimeUnit.MINUTES.toMicros(minutes) + TimeUnit.SECONDS.toMicros(seconds); + } + + private static long timeToNs(int hours, int minutes, int seconds) { + return TimeUnit.HOURS.toNanos(hours) + TimeUnit.MINUTES.toNanos(minutes) + TimeUnit.SECONDS.toNanos(seconds); + } + private void testDynamicWith(String withWhat, Object[] values, String[] expectedStrValues) throws Exception { if (isVersionMatch("(,24.8]")) { return; @@ -637,7 +828,8 @@ private void testDynamicWith(String withWhat, Object[] values, String[] expected String table = "test_dynamic_with_" + withWhat; client.execute("DROP TABLE IF EXISTS " + table).get(); client.execute(tableDefinition(table, "rowId Int32", "field Dynamic"), - (CommandSettings) new CommandSettings().serverSetting("allow_experimental_dynamic_type", "1")).get(); + (CommandSettings) new CommandSettings().serverSetting("allow_experimental_dynamic_type", "1") + .serverSetting("allow_experimental_time_time64_type", "1")).get(); client.register(DTOForDynamicPrimitivesTests.class, client.getTableSchema(table)); @@ -664,7 +856,9 @@ private void testVariantWith(String withWhat, String[] fields, Object[] values, System.arraycopy(fields, 0, actualFields, 1, fields.length); client.execute("DROP TABLE IF EXISTS " + table).get(); client.execute(tableDefinition(table, actualFields), - (CommandSettings) new CommandSettings().serverSetting("allow_experimental_variant_type", "1")).get(); + (CommandSettings) new CommandSettings() + .serverSetting("allow_experimental_variant_type", "1") + .serverSetting("allow_experimental_time_time64_type", "1")).get(); client.register(DTOForVariantPrimitivesTests.class, client.getTableSchema(table)); diff --git a/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java b/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java index 384639c25..46a949b38 100644 --- a/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java @@ -1334,8 +1334,10 @@ public void testNumberToStringConvertions() throws Exception { private static String sq(String str) { return "\'" + str + "\'"; } - void testDataTypes(List columns, List> valueGenerators, List> verifiers) { + testDataTypes(columns, valueGenerators, verifiers, Collections.emptyMap()); + } + void testDataTypes(List columns, List> valueGenerators, List> verifiers, Map serverSettings) { final String table = "data_types_test_table"; try { @@ -1343,6 +1345,10 @@ void testDataTypes(List columns, List> valueGenerators, client.execute("DROP TABLE IF EXISTS " + table).get(10, TimeUnit.SECONDS); // Create table + CommandSettings commandSettings = new CommandSettings(); + for (Map.Entry entry : serverSettings.entrySet()) { + commandSettings.serverSetting(entry.getKey(), entry.getValue()); + } StringBuilder createStmtBuilder = new StringBuilder(); createStmtBuilder.append("CREATE TABLE IF NOT EXISTS ").append(table).append(" ("); for (String column : columns) { @@ -1350,7 +1356,7 @@ void testDataTypes(List columns, List> valueGenerators, } createStmtBuilder.setLength(createStmtBuilder.length() - 2); createStmtBuilder.append(") ENGINE = MergeTree ORDER BY tuple()"); - client.execute(createStmtBuilder.toString()).get(10, TimeUnit.SECONDS); + client.execute(createStmtBuilder.toString(), commandSettings).get(10, TimeUnit.SECONDS); // Insert data diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java index 16e0b5890..8cb5002ac 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java @@ -41,10 +41,12 @@ import java.util.Random; import java.util.TimeZone; import java.util.UUID; +import java.util.concurrent.TimeUnit; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; @Test(groups = { "integration" }) @@ -550,6 +552,45 @@ public void testDateTypes() throws SQLException { } } + + @Test(groups = { "integration" }) + public void testTimeTypes() throws SQLException { + if (ClickHouseVersion.of(getServerVersion()).check("(,25.5]")) { + return; // Time64 introduced in 25.6 + } + Properties createProperties = new Properties(); + createProperties.put(ClientConfigProperties.serverSetting("allow_experimental_time_time64_type"), "1"); + runQuery("CREATE TABLE test_time64 (order Int8, " + + "time Time('UTC'), time64 Time64(9) " + + ") ENGINE = MergeTree ORDER BY ()", + createProperties); + + runQuery("INSERT INTO test_time64 (order, time, time64) VALUES " + + " (1, '-999:59:59', '-999:59:59.999999999'), " + + " (2, '999:59:59', '999:59:59.999999999')"); + + // Check the results + try (Statement stmt = getJdbcConnection().createStatement()) { + try (ResultSet rs = stmt.executeQuery("SELECT * FROM test_time64")) { + assertTrue(rs.next()); + assertEquals(rs.getInt("order"), 1); +// assertEquals(rs.getInt("time"), -(TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59)); +// assertEquals(rs.getInt("time64"), -(TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59)); + + assertTrue(rs.next()); + assertEquals(rs.getInt("order"), 2); + assertEquals(rs.getInt("time"), (TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59)); + assertEquals(rs.getLong("time64"), (TimeUnit.HOURS.toNanos(999) + TimeUnit.MINUTES.toNanos(59) + TimeUnit.SECONDS.toNanos(59)) + 999999999); + + assertThrows(SQLException.class, () -> rs.getTime("time")); + assertThrows(SQLException.class, () -> rs.getDate("time")); + assertThrows(SQLException.class, () -> rs.getTimestamp("time")); + + assertFalse(rs.next()); + } + } + } + @Test(groups = { "integration" }) public void testStringTypes() throws SQLException { runQuery("CREATE TABLE test_strings (order Int8, "