Skip to content

Commit a526116

Browse files
committed
adde support of Time and Time64 in the Client
1 parent 60ae503 commit a526116

File tree

8 files changed

+150
-7
lines changed

8 files changed

+150
-7
lines changed

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/data_formats/internal/AbstractBinaryFormatReader.java

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -301,6 +301,8 @@ protected void setSchema(TableSchema schema) {
301301
case Enum16:
302302
case Variant:
303303
case Dynamic:
304+
case Time:
305+
case Time64:
304306
this.convertions[i] = NumberConverter.NUMBER_CONVERTERS;
305307
break;
306308
default:
@@ -449,6 +451,9 @@ public Instant getInstant(String colName) {
449451
ZonedDateTime dateTime = (ZonedDateTime) colValue;
450452
return dateTime.toInstant();
451453
}
454+
case Time:
455+
case Time64:
456+
return readValue(colName);
452457
default:
453458
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
454459
}
@@ -631,7 +636,7 @@ public BigDecimal getBigDecimal(int index) {
631636

632637
@Override
633638
public Instant getInstant(int index) {
634-
return readValue(index);
639+
return getInstant(schema.columnIndexToName(index));
635640
}
636641

637642
@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
@@ -177,6 +177,10 @@ public <T> T readValue(ClickHouseColumn column, Class<?> typeHint) throws IOExce
177177
return convertDateTime(readDateTime32(timezone), typeHint);
178178
case DateTime64:
179179
return convertDateTime(readDateTime64(scale, timezone), typeHint);
180+
case Time:
181+
return (T) (Long) readUnsignedIntLE();
182+
case Time64:
183+
return (T) readBigIntegerLE(INT64_SIZE, true);
180184
case IntervalYear:
181185
case IntervalQuarter:
182186
case IntervalMonth:

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

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -130,6 +130,9 @@ public Instant getInstant(String colName) {
130130
case DateTime64:
131131
LocalDateTime dateTime = readValue(colName);
132132
return dateTime.toInstant(column.getTimeZone().toZoneId().getRules().getOffset(dateTime));
133+
case Time:
134+
return Instant.ofEpochSecond(getLong(colName));
135+
133136
}
134137
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
135138
}

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

Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -198,6 +198,8 @@ public static ClickHouseColumn valueToColumnForDynamicType(Object value) {
198198
column = enumValue2Column((Enum)value);
199199
} else if (value instanceof List<?> || (value !=null && value.getClass().isArray())) {
200200
column = listValue2Column(value);
201+
} else if (value instanceof Instant) {
202+
column = ClickHouseColumn.of("v", "Time64(9)");
201203
} else if (value == null) {
202204
column = PREDEFINED_TYPE_COLUMNS.get(Void.class);
203205
} else {
@@ -398,6 +400,10 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ
398400
case AggregateFunction:
399401
stream.write(binTag);
400402
break;
403+
case Time64:
404+
stream.write(binTag);
405+
BinaryStreamUtils.writeUnsignedInt8(stream, dt.getMaxPrecision());
406+
break;
401407
default:
402408
stream.write(binTag);
403409
}
@@ -545,6 +551,12 @@ private static void serializePrimitiveData(OutputStream stream, Object value, Cl
545551
writeDateTime64(stream, value, column.getScale(), zoneId);
546552
break;
547553
}
554+
case Time:
555+
BinaryStreamUtils.writeInt32(stream, convertToInteger(value));
556+
break;
557+
case Time64:
558+
serializeTime64(stream, value);
559+
break;
548560
case UUID:
549561
BinaryStreamUtils.writeUuid(stream, (UUID) value);
550562
break;
@@ -639,6 +651,19 @@ private static void serializeInterval(OutputStream stream, ClickHouseColumn colu
639651
BinaryStreamUtils.writeUnsignedInt64(stream, v);
640652
}
641653

654+
private static void serializeTime64(OutputStream stream, Object value) throws IOException {
655+
if (value instanceof BigInteger) {
656+
BinaryStreamUtils.writeUnsignedInt64(stream, (BigInteger) value);
657+
} else if (value instanceof Long) {
658+
BinaryStreamUtils.writeUnsignedInt64(stream, (Long) value);
659+
} else if (value instanceof Instant) {
660+
BinaryStreamUtils.writeUnsignedInt64(stream, BigInteger.valueOf(((Instant) value).getEpochSecond()).shiftLeft(32)
661+
.add(BigInteger.valueOf(((Instant) value).getNano())));
662+
} else {
663+
throw new UnsupportedOperationException("Cannot convert " + value.getClass() + " to Time64");
664+
}
665+
}
666+
642667
private static void serializeEnumData(OutputStream stream, ClickHouseColumn column, Object value) throws IOException {
643668
int enumValue = -1;
644669
if (value instanceof String) {

client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java

Lines changed: 95 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -3,13 +3,16 @@
33
import com.clickhouse.client.BaseIntegrationTest;
44
import com.clickhouse.client.ClickHouseNode;
55
import com.clickhouse.client.ClickHouseProtocol;
6+
import com.clickhouse.client.ClickHouseResponse;
67
import com.clickhouse.client.ClickHouseServerForTest;
78
import com.clickhouse.client.api.Client;
9+
import com.clickhouse.client.api.DataTypeUtils;
810
import com.clickhouse.client.api.command.CommandSettings;
911
import com.clickhouse.client.api.enums.Protocol;
1012
import com.clickhouse.client.api.insert.InsertSettings;
1113
import com.clickhouse.client.api.metadata.TableSchema;
1214
import com.clickhouse.client.api.query.GenericRecord;
15+
import com.clickhouse.client.api.query.QueryResponse;
1316
import com.clickhouse.data.ClickHouseDataType;
1417
import com.clickhouse.data.ClickHouseVersion;
1518
import lombok.AllArgsConstructor;
@@ -24,7 +27,10 @@
2427
import java.io.IOException;
2528
import java.lang.reflect.Method;
2629
import java.math.BigDecimal;
30+
import java.math.BigInteger;
31+
import java.sql.Time;
2732
import java.time.Duration;
33+
import java.time.Instant;
2834
import java.time.LocalDateTime;
2935
import java.time.Period;
3036
import java.time.temporal.ChronoUnit;
@@ -38,6 +44,7 @@
3844
import java.util.List;
3945
import java.util.Map;
4046
import java.util.Set;
47+
import java.util.concurrent.TimeUnit;
4148
import java.util.concurrent.atomic.AtomicInteger;
4249
import java.util.function.BiConsumer;
4350

@@ -189,6 +196,8 @@ public void testVariantWithSimpleDataTypes() throws Exception {
189196
case LowCardinality: // virtual type
190197
case LineString: // same as Ring
191198
case MultiLineString: // same as MultiPolygon
199+
case Time:
200+
case Time64:
192201
// tested separately
193202
continue dataTypesLoop;
194203

@@ -405,6 +414,29 @@ public void testVariantWithTuple() throws Exception {
405414
});
406415
}
407416

417+
@Test(groups = {"integration"})
418+
public void testVariantWithTimeTypes() throws Exception {
419+
testVariantWith("Time", new String[]{"field Variant(Time, String)"},
420+
new Object[]{
421+
"30:33:30",
422+
TimeUnit.HOURS.toSeconds(100) + TimeUnit.MINUTES.toSeconds(10) + 30
423+
},
424+
new String[]{
425+
"30:33:30",
426+
"360630", // Time stored as integer by default
427+
});
428+
429+
testVariantWith("Time64", new String[]{"field Variant(Time64, String)"},
430+
new Object[]{
431+
"30:33:30",
432+
TimeUnit.HOURS.toSeconds(100) + TimeUnit.MINUTES.toSeconds(10) + 30
433+
},
434+
new String[]{
435+
"30:33:30",
436+
"360630",
437+
});
438+
}
439+
408440
@Test(groups = {"integration"})
409441
public void testDynamicWithPrimitives() throws Exception {
410442
if (isVersionMatch("(,24.8]")) {
@@ -446,6 +478,8 @@ public void testDynamicWithPrimitives() throws Exception {
446478
case Enum: // virtual type
447479
case LineString: // same as Ring
448480
case MultiLineString: // same as MultiPolygon
481+
case Time:
482+
case Time64:
449483
// no tests or tested in other tests
450484
continue;
451485
default:
@@ -599,6 +633,29 @@ public void testDynamicWithMaps() throws Exception {
599633
});
600634
}
601635

636+
@Test(groups = {"integration"})
637+
public void testDynamicWithTimeTypes() throws Exception {
638+
long _999_hours = TimeUnit.HOURS.toSeconds(999);
639+
testDynamicWith("Time",
640+
new Object[]{
641+
_999_hours
642+
},
643+
new String[]{
644+
String.valueOf(_999_hours),
645+
});
646+
647+
Instant maxTime64 = Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59,
648+
999999999);
649+
650+
testDynamicWith("Time64",
651+
new Object[]{
652+
maxTime64,
653+
},
654+
new String[]{
655+
"3958241016481971977"
656+
});
657+
}
658+
602659
@Data
603660
@AllArgsConstructor
604661
public static class DTOForDynamicPrimitivesTests {
@@ -629,6 +686,39 @@ public void testAllDataTypesKnown() {
629686
Assert.assertTrue(unknowTypes.isEmpty(), "There are some unknown types: " + unknowTypes);
630687
}
631688

689+
@Test(groups = {"integration"})
690+
public void testTimeType() throws Exception {
691+
if (isVersionMatch("(,25.4]")) {
692+
return;
693+
}
694+
695+
String table = "test_time_type";
696+
client.execute("DROP TABLE IF EXISTS " + table).get();
697+
client.execute(tableDefinition(table, "o_num UInt32", "time Time"), (CommandSettings) new CommandSettings().serverSetting("enable_time_time64_type", "1")).get();
698+
699+
String insertSQL = "INSERT INTO " + table + " VALUES (1, '999:00:00'), (2, '999:59:00'), (3, '000:00:00')";
700+
try (QueryResponse response = client.query(insertSQL).get()) {}
701+
702+
703+
List<GenericRecord> records = client.queryAll("SELECT * FROM " + table);
704+
705+
GenericRecord record = records.get(0);
706+
Assert.assertEquals(record.getInteger("o_num"), 1);
707+
Assert.assertEquals(record.getInteger("time"), TimeUnit.HOURS.toSeconds(999));
708+
Assert.assertEquals(record.getInstant("time"), Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(999)));
709+
710+
record = records.get(1);
711+
Assert.assertEquals(record.getInteger("o_num"), 2);
712+
Assert.assertEquals(record.getInteger("time"), TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59));
713+
Assert.assertEquals(record.getInstant("time"), Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59)));
714+
715+
record = records.get(2);
716+
Assert.assertEquals(record.getInteger("o_num"), 3);
717+
Assert.assertEquals(record.getInteger("time"), 0);
718+
Assert.assertEquals(record.getInstant("time"), Instant.ofEpochSecond(0));
719+
}
720+
721+
632722
private void testDynamicWith(String withWhat, Object[] values, String[] expectedStrValues) throws Exception {
633723
if (isVersionMatch("(,24.8]")) {
634724
return;
@@ -637,7 +727,8 @@ private void testDynamicWith(String withWhat, Object[] values, String[] expected
637727
String table = "test_dynamic_with_" + withWhat;
638728
client.execute("DROP TABLE IF EXISTS " + table).get();
639729
client.execute(tableDefinition(table, "rowId Int32", "field Dynamic"),
640-
(CommandSettings) new CommandSettings().serverSetting("allow_experimental_dynamic_type", "1")).get();
730+
(CommandSettings) new CommandSettings().serverSetting("allow_experimental_dynamic_type", "1")
731+
.serverSetting("enable_time_time64_type", "1")).get();
641732

642733
client.register(DTOForDynamicPrimitivesTests.class, client.getTableSchema(table));
643734

@@ -664,7 +755,9 @@ private void testVariantWith(String withWhat, String[] fields, Object[] values,
664755
System.arraycopy(fields, 0, actualFields, 1, fields.length);
665756
client.execute("DROP TABLE IF EXISTS " + table).get();
666757
client.execute(tableDefinition(table, actualFields),
667-
(CommandSettings) new CommandSettings().serverSetting("allow_experimental_variant_type", "1")).get();
758+
(CommandSettings) new CommandSettings()
759+
.serverSetting("allow_experimental_variant_type", "1")
760+
.serverSetting("enable_time_time64_type", "1")).get();
668761

669762
client.register(DTOForVariantPrimitivesTests.class, client.getTableSchema(table));
670763

client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1304,23 +1304,29 @@ public void testNumberToStringConvertions() throws Exception {
13041304
private static String sq(String str) {
13051305
return "\'" + str + "\'";
13061306
}
1307-
13081307
void testDataTypes(List<String> columns, List<Supplier<String>> valueGenerators, List<Consumer<ClickHouseBinaryFormatReader>> verifiers) {
1308+
testDataTypes(columns, valueGenerators, verifiers, Collections.emptyMap());
1309+
}
1310+
void testDataTypes(List<String> columns, List<Supplier<String>> valueGenerators, List<Consumer<ClickHouseBinaryFormatReader>> verifiers, Map<String, String> serverSettings) {
13091311
final String table = "data_types_test_table";
13101312

13111313
try {
13121314
// Drop table
13131315
client.execute("DROP TABLE IF EXISTS " + table).get(10, TimeUnit.SECONDS);
13141316

13151317
// Create table
1318+
CommandSettings commandSettings = new CommandSettings();
1319+
for (Map.Entry<String, String> entry : serverSettings.entrySet()) {
1320+
commandSettings.serverSetting(entry.getKey(), entry.getValue());
1321+
}
13161322
StringBuilder createStmtBuilder = new StringBuilder();
13171323
createStmtBuilder.append("CREATE TABLE IF NOT EXISTS ").append(table).append(" (");
13181324
for (String column : columns) {
13191325
createStmtBuilder.append(column).append(", ");
13201326
}
13211327
createStmtBuilder.setLength(createStmtBuilder.length() - 2);
13221328
createStmtBuilder.append(") ENGINE = MergeTree ORDER BY tuple()");
1323-
client.execute(createStmtBuilder.toString()).get(10, TimeUnit.SECONDS);
1329+
client.execute(createStmtBuilder.toString(), commandSettings).get(10, TimeUnit.SECONDS);
13241330

13251331

13261332
// Insert data

0 commit comments

Comments
 (0)