Skip to content

Commit 2715492

Browse files
committed
added more type tests and fixed convertions to instant
1 parent 86acf74 commit 2715492

File tree

6 files changed

+167
-63
lines changed

6 files changed

+167
-63
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
}

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,7 +1,13 @@
11
package com.clickhouse.client.api;
22

3+
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
4+
5+
import java.time.Instant;
6+
import java.time.ZoneId;
37
import java.time.format.DateTimeFormatter;
48

9+
import static com.clickhouse.client.api.data_formats.internal.BinaryStreamReader.BASES;
10+
511
public class DataTypeUtils {
612

713
/**
@@ -19,4 +25,21 @@ public class DataTypeUtils {
1925
*/
2026
public static DateTimeFormatter DATETIME_WITH_NANOS_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.nnnnnnnnn");
2127

28+
public static Instant instantFromTime64Integer(int precision, long value) {
29+
int nanoSeconds = 0;
30+
if (precision > 0) {
31+
int factor = BinaryStreamReader.BASES[precision];
32+
nanoSeconds = (int) (value % factor);
33+
value /= factor;
34+
if (nanoSeconds < 0) {
35+
nanoSeconds += factor;
36+
value--;
37+
}
38+
if (nanoSeconds > 0L) {
39+
nanoSeconds *= BASES[9 - precision];
40+
}
41+
}
42+
43+
return Instant.ofEpochSecond(value, nanoSeconds);
44+
}
2245
}

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

Lines changed: 10 additions & 20 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;
@@ -455,8 +456,9 @@ public Instant getInstant(String colName) {
455456
return dateTime.toInstant();
456457
}
457458
case Time:
459+
return Instant.ofEpochSecond(getLong(colName));
458460
case Time64:
459-
return readValue(colName);
461+
return DataTypeUtils.instantFromTime64Integer(column.getScale(), getLong(colName));
460462
default:
461463
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
462464
}
@@ -711,22 +713,22 @@ public UUID getUUID(int index) {
711713

712714
@Override
713715
public ClickHouseGeoPointValue getGeoPoint(int index) {
714-
return readValue(index);
716+
return getGeoPoint(schema.columnIndexToName(index));
715717
}
716718

717719
@Override
718720
public ClickHouseGeoRingValue getGeoRing(int index) {
719-
return readValue(index);
721+
return getGeoRing(schema.columnIndexToName(index));
720722
}
721723

722724
@Override
723725
public ClickHouseGeoPolygonValue getGeoPolygon(int index) {
724-
return readValue(index);
726+
return getGeoPolygon(schema.columnIndexToName(index));
725727
}
726728

727729
@Override
728730
public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) {
729-
return readValue(index);
731+
return getGeoMultiPolygon(schema.columnIndexToName(index));
730732
}
731733

732734
@Override
@@ -808,11 +810,7 @@ public LocalDate getLocalDate(String colName) {
808810

809811
@Override
810812
public LocalDate getLocalDate(int index) {
811-
Object value = readValue(index);
812-
if (value instanceof ZonedDateTime) {
813-
return ((ZonedDateTime) value).toLocalDate();
814-
}
815-
return (LocalDate) value;
813+
return getLocalDate(schema.columnIndexToName(index));
816814
}
817815

818816
@Override
@@ -826,11 +824,7 @@ public LocalDateTime getLocalDateTime(String colName) {
826824

827825
@Override
828826
public LocalDateTime getLocalDateTime(int index) {
829-
Object value = readValue(index);
830-
if (value instanceof ZonedDateTime) {
831-
return ((ZonedDateTime) value).toLocalDateTime();
832-
}
833-
return (LocalDateTime) value;
827+
return getLocalDateTime(schema.columnIndexToName(index));
834828
}
835829

836830
@Override
@@ -844,11 +838,7 @@ public OffsetDateTime getOffsetDateTime(String colName) {
844838

845839
@Override
846840
public OffsetDateTime getOffsetDateTime(int index) {
847-
Object value = readValue(index);
848-
if (value instanceof ZonedDateTime) {
849-
return ((ZonedDateTime) value).toOffsetDateTime();
850-
}
851-
return (OffsetDateTime) value;
841+
return getOffsetDateTime(schema.columnIndexToName(index));
852842
}
853843

854844
@Override

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -187,9 +187,9 @@ public <T> T readValue(ClickHouseColumn column, Class<?> typeHint) throws IOExce
187187
case DateTime64:
188188
return convertDateTime(readDateTime64(scale, timezone), typeHint);
189189
case Time:
190-
return (T) (Long) readUnsignedIntLE();
190+
return (T) (Integer) readIntLE();
191191
case Time64:
192-
return (T) readBigIntegerLE(INT64_SIZE, true);
192+
return (T) (Long) (readLongLE());
193193
case IntervalYear:
194194
case IntervalQuarter:
195195
case IntervalMonth:

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

Lines changed: 28 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;
@@ -132,6 +143,8 @@ public Instant getInstant(String colName) {
132143
return dateTime.toInstant(column.getTimeZone().toZoneId().getRules().getOffset(dateTime));
133144
case Time:
134145
return Instant.ofEpochSecond(getLong(colName));
146+
case Time64:
147+
return DataTypeUtils.instantFromTime64Integer(column.getScale(), getLong(colName));
135148

136149
}
137150
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
@@ -251,7 +264,7 @@ public boolean[] getBooleanArray(String colName) {
251264

252265
@Override
253266
public boolean hasValue(int colIndex) {
254-
return record.containsKey(schema.columnIndexToName(colIndex));
267+
return hasValue(schema.columnIndexToName(colIndex));
255268
}
256269

257270
@Override
@@ -296,27 +309,27 @@ public boolean getBoolean(int index) {
296309

297310
@Override
298311
public BigInteger getBigInteger(int index) {
299-
return readValue(index);
312+
return getBigInteger(schema.columnIndexToName(index));
300313
}
301314

302315
@Override
303316
public BigDecimal getBigDecimal(int index) {
304-
return readValue(index);
317+
return getBigDecimal(schema.columnIndexToName(index));
305318
}
306319

307320
@Override
308321
public Instant getInstant(int index) {
309-
return readValue(index);
322+
return getInstant(schema.columnIndexToName(index));
310323
}
311324

312325
@Override
313326
public ZonedDateTime getZonedDateTime(int index) {
314-
return readValue(index);
327+
return getZonedDateTime(schema.columnIndexToName(index));
315328
}
316329

317330
@Override
318331
public Duration getDuration(int index) {
319-
return readValue(index);
332+
return getDuration(schema.columnIndexToName(index));
320333
}
321334

322335
@Override
@@ -341,22 +354,22 @@ public UUID getUUID(int index) {
341354

342355
@Override
343356
public ClickHouseGeoPointValue getGeoPoint(int index) {
344-
return readValue(index);
357+
return getGeoPoint(schema.columnIndexToName(index));
345358
}
346359

347360
@Override
348361
public ClickHouseGeoRingValue getGeoRing(int index) {
349-
return readValue(index);
362+
return getGeoRing(schema.columnIndexToName(index));
350363
}
351364

352365
@Override
353366
public ClickHouseGeoPolygonValue getGeoPolygon(int index) {
354-
return readValue(index);
367+
return getGeoPolygon(schema.columnIndexToName(index));
355368
}
356369

357370
@Override
358371
public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) {
359-
return readValue(index);
372+
return getGeoMultiPolygon(schema.columnIndexToName(index));
360373
}
361374

362375
@Override
@@ -426,11 +439,7 @@ public short getEnum16(int index) {
426439

427440
@Override
428441
public LocalDate getLocalDate(int index) {
429-
Object value = readValue(index);
430-
if (value instanceof ZonedDateTime) {
431-
return ((ZonedDateTime) value).toLocalDate();
432-
}
433-
return (LocalDate) value;
442+
return getLocalDate(schema.columnIndexToName(index));
434443
}
435444

436445
@Override
@@ -454,11 +463,7 @@ public LocalDateTime getLocalDateTime(String colName) {
454463

455464
@Override
456465
public LocalDateTime getLocalDateTime(int index) {
457-
Object value = readValue(index);
458-
if (value instanceof ZonedDateTime) {
459-
return ((ZonedDateTime) value).toLocalDateTime();
460-
}
461-
return (LocalDateTime) value;
466+
return getLocalDateTime(schema.columnIndexToName(index));
462467
}
463468

464469
@Override
@@ -472,11 +477,7 @@ public OffsetDateTime getOffsetDateTime(String colName) {
472477

473478
@Override
474479
public OffsetDateTime getOffsetDateTime(int index) {
475-
Object value = readValue(index);
476-
if (value instanceof ZonedDateTime) {
477-
return ((ZonedDateTime) value).toOffsetDateTime();
478-
}
479-
return (OffsetDateTime) value;
480+
return getOffsetDateTime(schema.columnIndexToName(index));
480481
}
481482

482483
@Override

0 commit comments

Comments
 (0)