Skip to content

Commit 8a797c9

Browse files
authored
Merge pull request #2487 from ClickHouse/new_time_datatypes
[client-v2, jdbc-v2] Support for Time and Time64
2 parents d9fda17 + 8ac3c80 commit 8a797c9

File tree

11 files changed

+358
-57
lines changed

11 files changed

+358
-57
lines changed

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

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -211,6 +211,7 @@ private static ClickHouseColumn update(ClickHouseColumn column) {
211211
}
212212
break;
213213
case DateTime32:
214+
case Time:
214215
if (size > 0) {
215216
column.template = ClickHouseOffsetDateTimeValue.ofNull(
216217
column.scale,
@@ -219,6 +220,7 @@ private static ClickHouseColumn update(ClickHouseColumn column) {
219220
}
220221
break;
221222
case DateTime64:
223+
case Time64:
222224
if (size > 0) {
223225
column.scale = Integer.parseInt(column.parameters.get(0));
224226
}

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

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -5,6 +5,7 @@
55
import java.net.Inet4Address;
66
import java.net.Inet6Address;
77
import java.time.Duration;
8+
import java.time.Instant;
89
import java.time.LocalDate;
910
import java.time.LocalDateTime;
1011
import java.time.LocalTime;
@@ -70,7 +71,7 @@ public enum ClickHouseDataType {
7071
// https://docs.oracle.com/javase/tutorial/java/nutsandbolts/datatypes.html#PageTitle
7172
UInt32(UnsignedInteger.class, false, true, false, 4, 10, 0, 0, 0, false, 0x03, "INT UNSIGNED", "INTEGER UNSIGNED",
7273
"MEDIUMINT UNSIGNED"),
73-
Int64(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x0A,"BIGINT", "BIGINT SIGNED", "TIME"),
74+
Int64(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x0A,"BIGINT", "BIGINT SIGNED"),
7475
IntervalYear(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22),
7576
IntervalQuarter(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22),
7677
IntervalMonth(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22),
@@ -127,6 +128,8 @@ public enum ClickHouseDataType {
127128
AggregateFunction(String.class, true, true, false, 0, 0, 0, 0, 0, true),
128129
Variant(List.class, true, true, false, 0, 0, 0, 0, 0, true, 0x2A),
129130
Dynamic(Object.class, true, true, false, 0, 0, 0, 0, 0, true, 0x2B),
131+
Time(LocalDateTime.class, true, false, false, 4, 9, 0, 0, 9, false, 0x32), // 0x33 for Time(Timezone)
132+
Time64(LocalDateTime.class, true, false, false, 8, 9, 0, 0, 0, false, 0x34), // 0x35 for Time64(P, Timezone)
130133
;
131134

132135
public static final List<ClickHouseDataType> ORDERED_BY_RANGE_INT_TYPES =
@@ -237,6 +240,9 @@ static Map<ClickHouseDataType, Set<Class<?>>> dataTypeClassMap() {
237240
map.put(IntervalMicrosecond, timeIntervalClasses);
238241
map.put(IntervalNanosecond, timeIntervalClasses);
239242

243+
map.put(Time, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(Integer.class, Long.class, Instant.class))));
244+
map.put(Time64, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(Integer.class, Long.class, BigInteger.class, Instant.class))));
245+
240246
return map;
241247
}
242248

clickhouse-data/src/test/java/com/clickhouse/data/ClickHouseColumnTest.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -420,7 +420,8 @@ public boolean isWidenUnsignedTypes() {
420420
// skip advanced types
421421
if (type.isNested() || type == ClickHouseDataType.AggregateFunction
422422
|| type == ClickHouseDataType.SimpleAggregateFunction || type == ClickHouseDataType.Enum
423-
|| type == ClickHouseDataType.Nullable || type == ClickHouseDataType.BFloat16) {
423+
|| type == ClickHouseDataType.Nullable || type == ClickHouseDataType.BFloat16 ||
424+
type == ClickHouseDataType.Time || type == ClickHouseDataType.Time64) {
424425
continue;
425426
}
426427

client-v2/src/main/java/com/clickhouse/client/api/DataTypeUtils.java

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,9 @@
11
package com.clickhouse.client.api;
22

3+
import java.time.Instant;
4+
import java.time.ZoneId;
5+
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
6+
37
import java.time.Instant;
48
import java.time.ZoneId;
59
import java.time.format.DateTimeFormatter;
@@ -9,6 +13,8 @@
913

1014
import com.clickhouse.data.ClickHouseDataType;
1115

16+
import static com.clickhouse.client.api.data_formats.internal.BinaryStreamReader.BASES;
17+
1218
public class DataTypeUtils {
1319

1420
/**
@@ -113,4 +119,21 @@ private static String formatInstantDefault(Instant instant) {
113119
return INSTANT_FORMATTER.format(instant);
114120
}
115121

122+
public static Instant instantFromTime64Integer(int precision, long value) {
123+
int nanoSeconds = 0;
124+
if (precision > 0) {
125+
int factor = BinaryStreamReader.BASES[precision];
126+
nanoSeconds = (int) (value % factor);
127+
value /= factor;
128+
if (nanoSeconds < 0) {
129+
nanoSeconds += factor;
130+
value--;
131+
}
132+
if (nanoSeconds > 0L) {
133+
nanoSeconds *= BASES[9 - precision];
134+
}
135+
}
136+
137+
return Instant.ofEpochSecond(value, nanoSeconds);
138+
}
116139
}

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

Lines changed: 16 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,7 @@
22

33
import com.clickhouse.client.api.ClientConfigProperties;
44
import com.clickhouse.client.api.ClientException;
5+
import com.clickhouse.client.api.DataTypeUtils;
56
import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
67
import com.clickhouse.client.api.internal.MapUtils;
78
import com.clickhouse.client.api.internal.ServerSettings;
@@ -304,6 +305,8 @@ protected void setSchema(TableSchema schema) {
304305
case Enum16:
305306
case Variant:
306307
case Dynamic:
308+
case Time:
309+
case Time64:
307310
this.convertions[i] = NumberConverter.NUMBER_CONVERTERS;
308311
break;
309312
default:
@@ -452,6 +455,10 @@ public Instant getInstant(String colName) {
452455
ZonedDateTime dateTime = (ZonedDateTime) colValue;
453456
return dateTime.toInstant();
454457
}
458+
case Time:
459+
return Instant.ofEpochSecond(getLong(colName));
460+
case Time64:
461+
return DataTypeUtils.instantFromTime64Integer(column.getScale(), getLong(colName));
455462
default:
456463
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
457464
}
@@ -472,7 +479,7 @@ public ZonedDateTime getZonedDateTime(String colName) {
472479
case Date32:
473480
return readValue(colName);
474481
default:
475-
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
482+
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to ZonedDateTime");
476483
}
477484
}
478485

@@ -671,7 +678,7 @@ public BigDecimal getBigDecimal(int index) {
671678

672679
@Override
673680
public Instant getInstant(int index) {
674-
return readValue(index);
681+
return getInstant(schema.columnIndexToName(index));
675682
}
676683

677684
@Override
@@ -706,22 +713,22 @@ public UUID getUUID(int index) {
706713

707714
@Override
708715
public ClickHouseGeoPointValue getGeoPoint(int index) {
709-
return readValue(index);
716+
return getGeoPoint(schema.columnIndexToName(index));
710717
}
711718

712719
@Override
713720
public ClickHouseGeoRingValue getGeoRing(int index) {
714-
return readValue(index);
721+
return getGeoRing(schema.columnIndexToName(index));
715722
}
716723

717724
@Override
718725
public ClickHouseGeoPolygonValue getGeoPolygon(int index) {
719-
return readValue(index);
726+
return getGeoPolygon(schema.columnIndexToName(index));
720727
}
721728

722729
@Override
723730
public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) {
724-
return readValue(index);
731+
return getGeoMultiPolygon(schema.columnIndexToName(index));
725732
}
726733

727734
@Override
@@ -803,11 +810,7 @@ public LocalDate getLocalDate(String colName) {
803810

804811
@Override
805812
public LocalDate getLocalDate(int index) {
806-
Object value = readValue(index);
807-
if (value instanceof ZonedDateTime) {
808-
return ((ZonedDateTime) value).toLocalDate();
809-
}
810-
return (LocalDate) value;
813+
return getLocalDate(schema.columnIndexToName(index));
811814
}
812815

813816
@Override
@@ -821,11 +824,7 @@ public LocalDateTime getLocalDateTime(String colName) {
821824

822825
@Override
823826
public LocalDateTime getLocalDateTime(int index) {
824-
Object value = readValue(index);
825-
if (value instanceof ZonedDateTime) {
826-
return ((ZonedDateTime) value).toLocalDateTime();
827-
}
828-
return (LocalDateTime) value;
827+
return getLocalDateTime(schema.columnIndexToName(index));
829828
}
830829

831830
@Override
@@ -839,11 +838,7 @@ public OffsetDateTime getOffsetDateTime(String colName) {
839838

840839
@Override
841840
public OffsetDateTime getOffsetDateTime(int index) {
842-
Object value = readValue(index);
843-
if (value instanceof ZonedDateTime) {
844-
return ((ZonedDateTime) value).toOffsetDateTime();
845-
}
846-
return (OffsetDateTime) value;
841+
return getOffsetDateTime(schema.columnIndexToName(index));
847842
}
848843

849844
@Override

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

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -186,6 +186,10 @@ public <T> T readValue(ClickHouseColumn column, Class<?> typeHint) throws IOExce
186186
return convertDateTime(readDateTime32(timezone), typeHint);
187187
case DateTime64:
188188
return convertDateTime(readDateTime64(scale, timezone), typeHint);
189+
case Time:
190+
return (T) (Integer) readIntLE();
191+
case Time64:
192+
return (T) (Long) (readLongLE());
189193
case IntervalYear:
190194
case IntervalQuarter:
191195
case IntervalMonth:

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

Lines changed: 31 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -1,17 +1,28 @@
11
package com.clickhouse.client.api.data_formats.internal;
22

33
import com.clickhouse.client.api.ClientException;
4+
import com.clickhouse.client.api.DataTypeUtils;
45
import com.clickhouse.client.api.metadata.TableSchema;
56
import com.clickhouse.client.api.query.GenericRecord;
67
import com.clickhouse.client.api.query.NullValueException;
78
import com.clickhouse.data.ClickHouseColumn;
8-
import com.clickhouse.data.value.*;
9+
import com.clickhouse.data.value.ClickHouseBitmap;
10+
import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue;
11+
import com.clickhouse.data.value.ClickHouseGeoPointValue;
12+
import com.clickhouse.data.value.ClickHouseGeoPolygonValue;
13+
import com.clickhouse.data.value.ClickHouseGeoRingValue;
914

1015
import java.math.BigDecimal;
1116
import java.math.BigInteger;
1217
import java.net.Inet4Address;
1318
import java.net.Inet6Address;
14-
import java.time.*;
19+
import java.time.Duration;
20+
import java.time.Instant;
21+
import java.time.LocalDate;
22+
import java.time.LocalDateTime;
23+
import java.time.OffsetDateTime;
24+
import java.time.ZoneOffset;
25+
import java.time.ZonedDateTime;
1526
import java.time.temporal.TemporalAmount;
1627
import java.util.HashMap;
1728
import java.util.List;
@@ -130,6 +141,11 @@ public Instant getInstant(String colName) {
130141
case DateTime64:
131142
LocalDateTime dateTime = readValue(colName);
132143
return dateTime.toInstant(column.getTimeZone().toZoneId().getRules().getOffset(dateTime));
144+
case Time:
145+
return Instant.ofEpochSecond(getLong(colName));
146+
case Time64:
147+
return DataTypeUtils.instantFromTime64Integer(column.getScale(), getLong(colName));
148+
133149
}
134150
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
135151
}
@@ -248,7 +264,7 @@ public boolean[] getBooleanArray(String colName) {
248264

249265
@Override
250266
public boolean hasValue(int colIndex) {
251-
return record.containsKey(schema.columnIndexToName(colIndex));
267+
return hasValue(schema.columnIndexToName(colIndex));
252268
}
253269

254270
@Override
@@ -293,27 +309,27 @@ public boolean getBoolean(int index) {
293309

294310
@Override
295311
public BigInteger getBigInteger(int index) {
296-
return readValue(index);
312+
return getBigInteger(schema.columnIndexToName(index));
297313
}
298314

299315
@Override
300316
public BigDecimal getBigDecimal(int index) {
301-
return readValue(index);
317+
return getBigDecimal(schema.columnIndexToName(index));
302318
}
303319

304320
@Override
305321
public Instant getInstant(int index) {
306-
return readValue(index);
322+
return getInstant(schema.columnIndexToName(index));
307323
}
308324

309325
@Override
310326
public ZonedDateTime getZonedDateTime(int index) {
311-
return readValue(index);
327+
return getZonedDateTime(schema.columnIndexToName(index));
312328
}
313329

314330
@Override
315331
public Duration getDuration(int index) {
316-
return readValue(index);
332+
return getDuration(schema.columnIndexToName(index));
317333
}
318334

319335
@Override
@@ -338,22 +354,22 @@ public UUID getUUID(int index) {
338354

339355
@Override
340356
public ClickHouseGeoPointValue getGeoPoint(int index) {
341-
return readValue(index);
357+
return getGeoPoint(schema.columnIndexToName(index));
342358
}
343359

344360
@Override
345361
public ClickHouseGeoRingValue getGeoRing(int index) {
346-
return readValue(index);
362+
return getGeoRing(schema.columnIndexToName(index));
347363
}
348364

349365
@Override
350366
public ClickHouseGeoPolygonValue getGeoPolygon(int index) {
351-
return readValue(index);
367+
return getGeoPolygon(schema.columnIndexToName(index));
352368
}
353369

354370
@Override
355371
public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) {
356-
return readValue(index);
372+
return getGeoMultiPolygon(schema.columnIndexToName(index));
357373
}
358374

359375
@Override
@@ -423,11 +439,7 @@ public short getEnum16(int index) {
423439

424440
@Override
425441
public LocalDate getLocalDate(int index) {
426-
Object value = readValue(index);
427-
if (value instanceof ZonedDateTime) {
428-
return ((ZonedDateTime) value).toLocalDate();
429-
}
430-
return (LocalDate) value;
442+
return getLocalDate(schema.columnIndexToName(index));
431443
}
432444

433445
@Override
@@ -451,11 +463,7 @@ public LocalDateTime getLocalDateTime(String colName) {
451463

452464
@Override
453465
public LocalDateTime getLocalDateTime(int index) {
454-
Object value = readValue(index);
455-
if (value instanceof ZonedDateTime) {
456-
return ((ZonedDateTime) value).toLocalDateTime();
457-
}
458-
return (LocalDateTime) value;
466+
return getLocalDateTime(schema.columnIndexToName(index));
459467
}
460468

461469
@Override
@@ -469,11 +477,7 @@ public OffsetDateTime getOffsetDateTime(String colName) {
469477

470478
@Override
471479
public OffsetDateTime getOffsetDateTime(int index) {
472-
Object value = readValue(index);
473-
if (value instanceof ZonedDateTime) {
474-
return ((ZonedDateTime) value).toOffsetDateTime();
475-
}
476-
return (OffsetDateTime) value;
480+
return getOffsetDateTime(schema.columnIndexToName(index));
477481
}
478482

479483
@Override

0 commit comments

Comments
 (0)