Skip to content

[client-v2, jdbc-v2] Support for Time and Time64 #2487

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 11 commits into from
Jul 23, 2025
Original file line number Diff line number Diff line change
Expand Up @@ -211,6 +211,7 @@ private static ClickHouseColumn update(ClickHouseColumn column) {
}
break;
case DateTime32:
case Time:
if (size > 0) {
column.template = ClickHouseOffsetDateTimeValue.ofNull(
column.scale,
Expand All @@ -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));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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),
Expand Down Expand Up @@ -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<ClickHouseDataType> ORDERED_BY_RANGE_INT_TYPES =
Expand Down Expand Up @@ -237,6 +240,9 @@ static Map<ClickHouseDataType, Set<Class<?>>> 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;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}

Expand Down
Original file line number Diff line number Diff line change
@@ -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;
Expand All @@ -9,6 +13,8 @@

import com.clickhouse.data.ClickHouseDataType;

import static com.clickhouse.client.api.data_formats.internal.BinaryStreamReader.BASES;

public class DataTypeUtils {

/**
Expand Down Expand Up @@ -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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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:
Expand Down Expand Up @@ -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");
}
Expand All @@ -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");
}
}

Expand Down Expand Up @@ -671,7 +678,7 @@ public BigDecimal getBigDecimal(int index) {

@Override
public Instant getInstant(int index) {
return readValue(index);
return getInstant(schema.columnIndexToName(index));
}

@Override
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand All @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -186,6 +186,10 @@ public <T> 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:
Expand Down
Original file line number Diff line number Diff line change
@@ -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;
Expand Down Expand Up @@ -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");
}
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand All @@ -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
Expand Down
Loading
Loading