From 61d2b5ff6c675166c2389b47415788b4fea79fca Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 10 Sep 2025 10:54:09 -0700 Subject: [PATCH 1/6] added data type converter and fixed problem with IP address as string --- .../clickhouse/client/api/DataTypeUtils.java | 13 ++ .../internal/AbstractBinaryFormatReader.java | 51 +---- .../internal/MapBackedRecord.java | 7 +- .../api/internal/DataTypeConverter.java | 181 ++++++++++++++++++ .../api/internal/DataTypeConverterTest.java | 21 ++ .../client/datatypes/DataTypeTests.java | 27 +++ .../com/clickhouse/jdbc/DataTypeTests.java | 6 +- .../jdbc/PreparedStatementTest.java | 7 +- .../com/clickhouse/jdbc/StatementTest.java | 19 -- 9 files changed, 258 insertions(+), 74 deletions(-) create mode 100644 client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java create mode 100644 client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java 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 eed908202..ce62a86b4 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,6 @@ package com.clickhouse.client.api; +import java.text.SimpleDateFormat; import java.time.Instant; import java.time.ZoneId; import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader; @@ -39,6 +40,18 @@ public class DataTypeUtils { .appendFraction(ChronoField.NANO_OF_SECOND, 9, 9, true) .toFormatter(); + public static final DateTimeFormatter TIME_WITH_NANOS_FORMATTER = INSTANT_FORMATTER; + + public static final DateTimeFormatter TIME_FORMATTER = DateTimeFormatter.ofPattern("HH:mm:ss"); + + public static final SimpleDateFormat OLD_DATE_TIME_FORMATTER = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + + public static final SimpleDateFormat OLD_DATE_FORMATTER = new SimpleDateFormat("yyyy-MM-dd"); + + public static final SimpleDateFormat OLD_TIME_FORMATTER = new SimpleDateFormat("HH:mm:ss"); + + public static final SimpleDateFormat OLD_TIME_WITH_NANOS_FORMATTER = new SimpleDateFormat("HH:mm:ss.SSSSSSSSS"); + /** * Formats an {@link Instant} object for use in SQL statements or as query * parameter. 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 d393d8503..5d0d051c4 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 @@ -4,6 +4,7 @@ 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.DataTypeConverter; import com.clickhouse.client.api.internal.MapUtils; import com.clickhouse.client.api.internal.ServerSettings; import com.clickhouse.client.api.metadata.NoSuchColumnException; @@ -61,6 +62,8 @@ public abstract class AbstractBinaryFormatReader implements ClickHouseBinaryForm protected BinaryStreamReader binaryStreamReader; + protected DataTypeConverter dataTypeConverter; + private TableSchema schema; private ClickHouseColumn[] columns; private Map[] convertions; @@ -84,6 +87,7 @@ protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings quer if (schema != null) { setSchema(schema); } + this.dataTypeConverter = DataTypeConverter.INSTANCE; // singleton while no need to customize conversion } protected Object[] currentRecord; @@ -326,52 +330,7 @@ public TableSchema getSchema() { @Override public String getString(String colName) { - return readAsString(readValue(colName), schema.getColumnByName(colName)); - } - - /** - * Converts value in to a string representation. Does some formatting for selected data types - * @return string representation of a value for specified column - */ - public static String readAsString(Object value, ClickHouseColumn column) { - if (value == null) { - return null; - } else if (value instanceof String) { - return (String) value; - } else if (value instanceof ZonedDateTime) { - ClickHouseDataType dataType = column.getDataType(); - ZonedDateTime zdt = (ZonedDateTime) value; - switch (dataType) { // should not be null - case Date: - case Date32: - return zdt.format(DataTypeUtils.DATE_FORMATTER); - case DateTime: - case DateTime32: - return zdt.format(DataTypeUtils.DATETIME_FORMATTER); - case DateTime64: - return zdt.format(DataTypeUtils.DATETIME_WITH_NANOS_FORMATTER); - default: - return value.toString(); - } - } else if (value instanceof BinaryStreamReader.EnumValue) { - return ((BinaryStreamReader.EnumValue)value).name; - } else if (value instanceof Number ) { - ClickHouseDataType dataType = column.getDataType(); - int num = ((Number) value).intValue(); - if (column.getDataType() == ClickHouseDataType.Variant) { - for (ClickHouseColumn c : column.getNestedColumns()) { - // TODO: will work only if single enum listed as variant - if (c.getDataType() == ClickHouseDataType.Enum8 || c.getDataType() == ClickHouseDataType.Enum16) { - return c.getEnumConstants().name(num); - } - } - } else if (dataType == ClickHouseDataType.Enum8 || dataType == ClickHouseDataType.Enum16) { - return column.getEnumConstants().name(num); - } - } else if (value instanceof BinaryStreamReader.ArrayValue) { - return ((BinaryStreamReader.ArrayValue)value).asList().toString(); - } - return value.toString(); + return dataTypeConverter.convertToString(readValue(colName), schema.getColumnByName(colName)); } @Override 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 23bb5cd07..d881bca45 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 @@ -2,6 +2,7 @@ import com.clickhouse.client.api.ClientException; import com.clickhouse.client.api.DataTypeUtils; +import com.clickhouse.client.api.internal.DataTypeConverter; import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.GenericRecord; import com.clickhouse.client.api.query.NullValueException; @@ -24,6 +25,7 @@ import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.time.temporal.TemporalAmount; +import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -38,10 +40,13 @@ public class MapBackedRecord implements GenericRecord { private Map[] columnConverters; + private DataTypeConverter dataTypeConverter; + public MapBackedRecord(Map record, Map[] columnConverters, TableSchema schema) { this.record = new HashMap<>(record); this.schema = schema; this.columnConverters = columnConverters; + this.dataTypeConverter = DataTypeConverter.INSTANCE; } public T readValue(int colIndex) { @@ -58,7 +63,7 @@ public T readValue(String colName) { @Override public String getString(String colName) { - return AbstractBinaryFormatReader.readAsString(readValue(colName), schema.getColumnByName(colName)); + return dataTypeConverter.convertToString(readValue(colName), schema.getColumnByName(colName)); } @Override diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java new file mode 100644 index 000000000..9710c48a2 --- /dev/null +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java @@ -0,0 +1,181 @@ +package com.clickhouse.client.api.internal; + +import com.clickhouse.client.api.DataTypeUtils; +import com.clickhouse.client.api.data_formats.internal.AbstractBinaryFormatReader; +import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader; +import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.ClickHouseDataType; + +import java.net.InetAddress; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.time.temporal.TemporalAccessor; +import java.util.Arrays; +import java.util.Date; + +public class DataTypeConverter { + + public static final DataTypeConverter INSTANCE = new DataTypeConverter(); + + public String convertToString(Object value, ClickHouseColumn column) { + if (value == null) { + return null; + } + + switch (column.getDataType()) { + case String: + return stringToString(value, column); + case Date: + case Date32: + return dateToString(value, column); + case Time: + case Time64: + return timeToString(value, column); + case DateTime: + case DateTime32: + case DateTime64: + return dateTimeToString(value, column); + case Enum8: + case Enum16: + case Enum: + return enumToString(value, column); + case IPv4: + case IPv6: + return ipvToString(value, column); + case Array: + return arrayToString(value, column); + case Variant: + case Dynamic: + return variantOrDynamicToString(value, column); + default: + return value.toString(); + } + } + + public String stringToString(Object bytesOrString, ClickHouseColumn column) { + return bytesOrString instanceof byte[] ? new String((byte[]) bytesOrString) : (String) bytesOrString; + } + + public String dateToString(Object value, ClickHouseColumn column) { + DateTimeFormatter formatter = DataTypeUtils.DATE_FORMATTER; + + if (value instanceof ZonedDateTime || value instanceof LocalDateTime) { + TemporalAccessor dateTime = (TemporalAccessor) value; + return formatter.format(dateTime); + } else if (value instanceof LocalDate) { + return formatter.format(((LocalDate)value)); + } else if (value instanceof Date) { + return DataTypeUtils.OLD_DATE_FORMATTER.format(((Date)value)); + } + return value.toString(); + } + + public String timeToString(Object value, ClickHouseColumn column) { + DateTimeFormatter formatter; + switch (column.getDataType()) { + case Time64: + formatter = DataTypeUtils.TIME_WITH_NANOS_FORMATTER; + break; + default: + formatter = DataTypeUtils.TIME_FORMATTER; + } + + if (value instanceof ZonedDateTime || value instanceof LocalDateTime) { + TemporalAccessor dateTime = (TemporalAccessor) value; + return formatter.format(dateTime); + } else if (value instanceof LocalTime) { + return formatter.format(((LocalTime)value)); + } else if (value instanceof Date) { + return DataTypeUtils.OLD_TIME_FORMATTER.format(((Date)value)); + } + return value.toString(); + } + + public String dateTimeToString(Object value, ClickHouseColumn column) { + DateTimeFormatter formatter; + switch (column.getDataType()) { + case DateTime64: + formatter = DataTypeUtils.DATETIME_WITH_NANOS_FORMATTER; + break; + default: + formatter = DataTypeUtils.DATETIME_FORMATTER; + } + + if (value instanceof ZonedDateTime || value instanceof LocalDateTime) { + TemporalAccessor dateTime = (TemporalAccessor) value; + return formatter.format(dateTime); + } else if (value instanceof LocalDate) { + return formatter.format(((LocalDate)value).atStartOfDay()); + } else if (value instanceof LocalTime) { + return formatter.format(((LocalTime)value).atDate(LocalDate.now())); + } else if (value instanceof Date) { + return DataTypeUtils.OLD_DATE_TIME_FORMATTER.format(((Date)value)); + } + return value.toString(); + } + + public String enumToString(Object value, ClickHouseColumn column) { + if (value instanceof BinaryStreamReader.EnumValue) { + return ((BinaryStreamReader.EnumValue)value).name; + } else if (value instanceof Number ) { + int num = ((Number) value).intValue(); + switch (column.getDataType()) { + case Variant: + for (ClickHouseColumn c : column.getNestedColumns()) { + // TODO: will work only if single enum listed as variant + if (c.getDataType() == ClickHouseDataType.Enum8 || c.getDataType() == ClickHouseDataType.Enum16) { + return c.getEnumConstants().name(num); + } + } + return String.valueOf(num); // fail-safe + case Enum8: + case Enum16: + case Enum: + return column.getEnumConstants().name(num); + } + } + return value.toString(); + } + + public String ipvToString(Object value, ClickHouseColumn column) { + if (value instanceof InetAddress) { + return ((InetAddress) value).getHostAddress(); + } + return value.toString(); + } + + public String arrayToString(Object value, ClickHouseColumn column) { + if (value instanceof BinaryStreamReader.ArrayValue) { + return ((BinaryStreamReader.ArrayValue) value).asList().toString(); + } else if (value instanceof byte[]) { + return Arrays.toString((byte[]) value); + } else if (value instanceof short[]) { + return Arrays.toString((short[]) value); + } else if (value instanceof int[]) { + return Arrays.toString((int[]) value); + } else if (value instanceof long[]) { + return Arrays.toString((long[]) value); + } else if (value instanceof float[]) { + return Arrays.toString((float[]) value); + } else if (value instanceof double[]) { + return Arrays.toString((double[]) value); + } else if (value instanceof boolean[]) { + return Arrays.toString((boolean[]) value); + } else if (value instanceof char[]) { + return Arrays.toString((char[]) value); + } else if (value instanceof Object[]) { + return Arrays.deepToString((Object[]) value); + } + return value.toString(); + } + + public String variantOrDynamicToString(Object value, ClickHouseColumn column) { + if (value instanceof BinaryStreamReader.ArrayValue) { + return arrayToString(value, column); + } + return value.toString(); + } +} diff --git a/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java b/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java new file mode 100644 index 000000000..cf3e6ec9a --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java @@ -0,0 +1,21 @@ +package com.clickhouse.client.api.internal; + +import org.testng.annotations.Test; + +import static org.testng.Assert.*; + +@Test(groups = {"unit"}) +public class DataTypeConverterTest { + + @Test + public void testDateToString() { + } + + @Test + public void testTimeToString() { + } + + @Test + public void testDateTimeToString() { + } +} \ No newline at end of file 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 3b66d4c7c..0b2287e9d 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 @@ -916,6 +916,33 @@ public Object[][] testJSONBinaryFormat_dp() { }; } + @Test(groups = {"integration"}, dataProvider = "testDataTypesAsStringDP") + public void testDataTypesAsString(String sql, String[] expectedStrValues) throws Exception { + + try (QueryResponse resp = client.query(sql).get()) { + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(resp); + reader.next(); + for (int i = 0; i < expectedStrValues.length; i++) { + Assert.assertEquals(reader.getString(i + 1), expectedStrValues[i]); + } + } + } + + @DataProvider + public static Object[][] testDataTypesAsStringDP() { + return new Object[][] { + {"SELECT '192.168.1.1'::IPv4, '2001:db8::1'::IPv6, '192.168.1.1'::IPv6", + new String[] {"192.168.1.1", "2001:db8:0:0:0:0:0:1", "192.168.1.1"}}, + {"SELECT '2024-10-04'::Date32, '2024-10-04 12:34:56'::DateTime32, '2024-10-04 12:34:56.789'::DateTime64(3), " + + " '2024-10-04 12:34:56.789012'::DateTime64(6), '2024-10-04 12:34:56.789012345'::DateTime64(9)", + new String[] {"2024-10-04", "2024-10-04 12:34:56", "2024-10-04 12:34:56.789", "2024-10-04 12:34:56.789012", + "2024-10-04 12:34:56.789012345"}}, + {"SELECT 1::Enum16('one' = 1, 'two' = 2)", "one"}, + {"SELECT 2::Enum8('one' = 1, 'two' = 2)", "two"}, + {"SELECT 3::Enum('one' = 1, 'two' = 2, 'three' = 3)", "three"}, + }; + } + public static String tableDefinition(String table, String... columns) { StringBuilder sb = new StringBuilder(); sb.append("CREATE TABLE " + table + " ( "); 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 bb958adb4..892065a32 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java @@ -749,11 +749,11 @@ public void testIpAddressTypes() throws SQLException, UnknownHostException { try (ResultSet rs = stmt.executeQuery("SELECT * FROM test_ips ORDER BY order")) { assertTrue(rs.next()); assertEquals(rs.getObject("ipv4_ip"), ipv4AddressByIp); - assertEquals(rs.getObject("ipv4_ip", Inet6Address.class).toString(), "/0:0:0:0:0:ffff:5ab0:4b61"); - assertEquals(rs.getString("ipv4_ip"), ipv4AddressByIp.toString()); + assertEquals(rs.getObject("ipv4_ip", Inet6Address.class).getHostAddress(), "0:0:0:0:0:ffff:5ab0:4b61"); + assertEquals(rs.getString("ipv4_ip"), ipv4AddressByIp.getHostAddress()); assertEquals(rs.getObject("ipv4_name"), ipv4AddressByName); assertEquals(rs.getObject("ipv6"), ipv6Address); - assertEquals(rs.getString("ipv6"), ipv6Address.toString()); + assertEquals(rs.getString("ipv6"), ipv6Address.getHostAddress()); assertEquals(rs.getObject("ipv4_as_ipv6"), ipv4AsIpv6); assertEquals(rs.getObject("ipv4_as_ipv6", Inet4Address.class), ipv4AsIpv6); assertFalse(rs.next()); diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/PreparedStatementTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/PreparedStatementTest.java index b3cda1247..8d8f3e2fe 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/PreparedStatementTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/PreparedStatementTest.java @@ -1388,11 +1388,8 @@ public void testTypeCastsWithoutArgument(Object value, SQLType targetType, Click rs.next(); assertEquals(rs.getString(2), expectedType.getName()); switch (expectedType) { - case IPv4: - assertEquals(rs.getString(1), "/" + value); - break; case IPv6: - // do not check + // do not check because auto-converted to IPv4 break; default: assertEquals(rs.getString(1), String.valueOf(value)); @@ -1414,7 +1411,7 @@ public static Object[][] testTypeCastsDP() { {100L, ClickHouseDataType.UInt32, ClickHouseDataType.UInt32}, {100L, ClickHouseDataType.UInt64, ClickHouseDataType.UInt64}, {"ed0c77a3-2e4b-4954-98ee-22a4fdad9565", ClickHouseDataType.UUID, ClickHouseDataType.UUID}, - {"::ffff:127.0.0.1", ClickHouseDataType.IPv6, ClickHouseDataType.IPv6}, + {"0:0:0:0:0:ffff:5ab0:4b61", ClickHouseDataType.IPv6, ClickHouseDataType.IPv6}, {"116.253.40.133", ClickHouseDataType.IPv4, ClickHouseDataType.IPv4}, {100, JDBCType.TINYINT, ClickHouseDataType.Int8} }; diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java index 2a562bae5..7f8c65de7 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java @@ -489,25 +489,6 @@ public void testGettingArrays() throws Exception { } } - - @Test(groups = {"integration"}) - public void testWithIPs() throws Exception { - try (Connection conn = getJdbcConnection()) { - try (Statement stmt = conn.createStatement()) { - try (ResultSet rs = stmt.executeQuery("SELECT toIPv4('127.0.0.1'), toIPv6('::1'), toIPv6('2001:438:ffff::407d:1bc1')")) { - assertTrue(rs.next()); - assertEquals(rs.getString(1), "/127.0.0.1"); - assertEquals(rs.getObject(1), Inet4Address.getByName("127.0.0.1")); - assertEquals(rs.getString(2), "/0:0:0:0:0:0:0:1"); - assertEquals(rs.getObject(2), Inet6Address.getByName("0:0:0:0:0:0:0:1")); - assertEquals(rs.getString(3), "/2001:438:ffff:0:0:0:407d:1bc1"); - assertEquals(rs.getObject(3), Inet6Address.getByName("2001:438:ffff:0:0:0:407d:1bc1")); - assertFalse(rs.next()); - } - } - } - } - @Test public void testConnectionExhaustion() throws Exception { From 4e6a6bd1d039409e8095a4a7d12ef528198b4caf Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Mon, 22 Sep 2025 18:46:45 -0700 Subject: [PATCH 2/6] implemnted base collection writer to iterate over nested arrays and collections --- .../api/internal/BaseCollectionConverter.java | 217 ++++++++++++++++++ .../api/internal/DataTypeConverter.java | 132 +++++++++-- .../api/internal/DataTypeConverterTest.java | 38 ++- .../client/datatypes/DataTypeTests.java | 4 +- 4 files changed, 361 insertions(+), 30 deletions(-) create mode 100644 client-v2/src/main/java/com/clickhouse/client/api/internal/BaseCollectionConverter.java diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/BaseCollectionConverter.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/BaseCollectionConverter.java new file mode 100644 index 000000000..f051b8f34 --- /dev/null +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/BaseCollectionConverter.java @@ -0,0 +1,217 @@ +package com.clickhouse.client.api.internal; + +import com.clickhouse.client.api.ClickHouseException; + +import java.io.IOException; +import java.lang.reflect.Array; +import java.util.List; +import java.util.Stack; + +public abstract class BaseCollectionConverter { + public static final String ARRAY_START = "["; + public static final String ARRAY_END = "]"; + + private final String itemDelimiter; + + protected BaseCollectionConverter(String itemDelimiter) { + this.itemDelimiter = itemDelimiter; + } + + protected abstract void setAccumulator(TAcc acc); + + protected abstract void append(String str); + + protected abstract String buildString(); + + protected abstract void onStart(ListConversionState state); + + protected abstract void onEnd(ListConversionState state); + + protected abstract void onItem(Object item, ListConversionState state); + + protected abstract String onEmptyCollection(); + + protected abstract boolean isEmpty(TList list); + + protected abstract boolean isSubList(Object list); + + protected abstract int listSize(TList list); + + protected abstract Object getNext(ListConversionState state); + + public final String convert(TList value, TAcc acc) { + if (isEmpty(value)) { + return onEmptyCollection(); + } + setAccumulator(acc); + + Stack> stack = new Stack<>(); + ListConversionState state = new ListConversionState<>(value, listSize(value)); + while (state != null) { + if (state.isFirst()) { + onStart(state); + } + if (state.hasNext()) { + Object item = getNext(state); + state.incPos(); + if (isSubList(item)) { + stack.push(state); + TList list = (TList) item; + state = new ListConversionState<>(list, listSize(list)); + } else { + onItem(item, state); + if (state.hasNext()) { + append(itemDelimiter); + } + } + } else { + onEnd(state); + state = stack.isEmpty() ? null : stack.pop(); + if (state != null && state.hasNext()) { + append(itemDelimiter); + } + } + } + + return buildString(); + } + + public static final class ListConversionState { + + final TList list; + int position; + int size; + + public ListConversionState(TList list, int size) { + this.list = list; + this.position = 0; + this.size = size; + } + + public TList getList() { + return list; + } + + public int getPosition() { + return position; + } + + public void incPos() { + this.position++; + } + + public boolean hasNext() { + return position < size; + } + + public boolean isFirst() { + return position == 0; + } + } + + public static abstract class BaseArrayWriter extends BaseCollectionWriter { + + protected BaseArrayWriter() { + super(", "); + } + + @Override + protected boolean isEmpty(Object objects) { + return listSize(objects) == 0; + } + + @Override + protected boolean isSubList(Object list) { + return list != null && list.getClass().isArray(); + } + + @Override + protected int listSize(Object objects) { + return Array.getLength(objects); + } + + @Override + protected Object getNext(ListConversionState state) { + return Array.get(state.getList(), state.getPosition()); + } + } + + public static abstract class BaseListWriter + extends BaseCollectionWriter> { + public BaseListWriter() { + super(", "); + } + + @Override + protected boolean isEmpty(List objects) { + return objects.isEmpty(); + } + + @Override + protected boolean isSubList(Object list) { + return list instanceof List; + } + + @Override + protected int listSize(List objects) { + return objects.size(); + } + + @Override + protected Object getNext(ListConversionState> state) { + return state.getList().get(state.getPosition()); + } + } + + public static abstract class BaseCollectionWriter extends + BaseCollectionConverter { + + protected Appendable appendable; + + protected BaseCollectionWriter(String itemDelimiter) { + super(itemDelimiter); + } + + @Override + protected void setAccumulator(Appendable appendable) { + this.appendable = appendable; + } + + @Override + protected void append(String str) { + try { + appendable.append(str); + } catch (IOException e) { + throw new ClickHouseException(e.getMessage(), e); + } + } + + @Override + protected String buildString() { + return appendable.toString(); + } + + @Override + protected void onStart(ListConversionState state) { + try { + appendable.append(ARRAY_START); + } catch (IOException e) { + throw new ClickHouseException(e.getMessage(), e); + } + } + + @Override + protected void onEnd(ListConversionState state) { + try { + appendable.append(ARRAY_END); + } catch (IOException e) { + throw new ClickHouseException(e.getMessage(), e); + } + } + + @Override + protected String onEmptyCollection() { + return ARRAY_START + ARRAY_END; + } + } +} diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java index 9710c48a2..3f903c829 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java @@ -1,7 +1,7 @@ package com.clickhouse.client.api.internal; +import com.clickhouse.client.api.ClickHouseException; import com.clickhouse.client.api.DataTypeUtils; -import com.clickhouse.client.api.data_formats.internal.AbstractBinaryFormatReader; import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader; import com.clickhouse.data.ClickHouseColumn; import com.clickhouse.data.ClickHouseDataType; @@ -13,13 +13,26 @@ import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; import java.time.temporal.TemporalAccessor; -import java.util.Arrays; import java.util.Date; +import java.util.List; +/** + * Class designed to convert different data types to Java objects. + * First use-case is to convert ClickHouse data types to String representation. + * Note: class is not thread-safe to avoid extra object creation. + */ public class DataTypeConverter { + private static final char QUOTE = '\''; + + private static final String NULL = "NULL"; + public static final DataTypeConverter INSTANCE = new DataTypeConverter(); + private final ListAsStringWriter listAsStringWriter = new ListAsStringWriter(); + + private final ArrayAsStringWriter arrayAsStringWriter = new ArrayAsStringWriter(); + public String convertToString(Object value, ClickHouseColumn column) { if (value == null) { return null; @@ -56,7 +69,21 @@ public String convertToString(Object value, ClickHouseColumn column) { } public String stringToString(Object bytesOrString, ClickHouseColumn column) { - return bytesOrString instanceof byte[] ? new String((byte[]) bytesOrString) : (String) bytesOrString; + StringBuilder sb = new StringBuilder(); + if (column.isArray()) { + sb.append(QUOTE); + } + if (bytesOrString instanceof CharSequence) { + sb.append(((CharSequence) bytesOrString)); + } else if (bytesOrString instanceof byte[]) { + sb.append(bytesOrString); + } else { + sb.append(bytesOrString); + } + if (column.isArray()) { + sb.append(QUOTE); + } + return sb.toString(); } public String dateToString(Object value, ClickHouseColumn column) { @@ -149,33 +176,94 @@ public String ipvToString(Object value, ClickHouseColumn column) { public String arrayToString(Object value, ClickHouseColumn column) { if (value instanceof BinaryStreamReader.ArrayValue) { - return ((BinaryStreamReader.ArrayValue) value).asList().toString(); - } else if (value instanceof byte[]) { - return Arrays.toString((byte[]) value); - } else if (value instanceof short[]) { - return Arrays.toString((short[]) value); - } else if (value instanceof int[]) { - return Arrays.toString((int[]) value); - } else if (value instanceof long[]) { - return Arrays.toString((long[]) value); - } else if (value instanceof float[]) { - return Arrays.toString((float[]) value); - } else if (value instanceof double[]) { - return Arrays.toString((double[]) value); - } else if (value instanceof boolean[]) { - return Arrays.toString((boolean[]) value); - } else if (value instanceof char[]) { - return Arrays.toString((char[]) value); - } else if (value instanceof Object[]) { - return Arrays.deepToString((Object[]) value); + return listAsStringWriter.convertAndReset(((BinaryStreamReader.ArrayValue) value).asList(), new StringBuilder(), column); + } else if (value.getClass().isArray()) { + return arrayAsStringWriter.convertAndReset(value, new StringBuilder(), column); + } else if (value instanceof List) { + return listAsStringWriter.convertAndReset((List) value, new StringBuilder(), column); } return value.toString(); } + /** + * + * @param value not null object value to convert + * @param column column describing the DB value + * @return String representing the value + */ public String variantOrDynamicToString(Object value, ClickHouseColumn column) { if (value instanceof BinaryStreamReader.ArrayValue) { return arrayToString(value, column); } return value.toString(); } + + private final class ArrayAsStringWriter extends BaseCollectionConverter.BaseArrayWriter { + private ClickHouseColumn column; + + ArrayAsStringWriter() { + super(); + } + + public void setColumn(ClickHouseColumn column) { + this.column = column; + } + + + @Override + protected void onItem(Object item, ListConversionState state) { + if (item == null) { + append(NULL); + return; + } + String str = DataTypeConverter.this.convertToString(item, column.getArrayBaseColumn() == null ? column : column.getArrayBaseColumn()); + try { + if (column.getArrayBaseColumn().getDataType() == ClickHouseDataType.String) { + appendable.append(QUOTE).append(str).append(QUOTE); + } else { + appendable.append(str); + } + } catch (Exception ex) { + throw new ClickHouseException(ex.getMessage(), ex); + } + } + + public String convertAndReset(Object list, Appendable acc, ClickHouseColumn column) { + try { + setColumn(column); + return super.convert(list, acc); + } finally { + this.column = null; + setAccumulator(null); + } + } + } + + private final class ListAsStringWriter extends BaseCollectionConverter.BaseListWriter { + + private ClickHouseColumn column; + + public void setColumn(ClickHouseColumn column) { + this.column = column; + } + + @Override + protected void onItem(Object item, ListConversionState> state) { + if (item == null) { + append(NULL); + return; + } + append(DataTypeConverter.this.convertToString(item, column.getArrayBaseColumn() == null ? column : column.getArrayBaseColumn())); + } + + public String convertAndReset(List list, Appendable acc, ClickHouseColumn column) { + try { + setColumn(column); + return super.convert(list, acc); + } finally { + this.column = null; + setAccumulator(null); + } + } + } } diff --git a/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java b/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java index cf3e6ec9a..87759882f 100644 --- a/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java +++ b/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java @@ -1,21 +1,47 @@ package com.clickhouse.client.api.internal; +import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.ClickHouseDataType; import org.testng.annotations.Test; -import static org.testng.Assert.*; +import java.util.Arrays; +import java.util.Collections; + +import static org.testng.Assert.assertEquals; @Test(groups = {"unit"}) public class DataTypeConverterTest { @Test - public void testDateToString() { - } + public void testArrayToString() { + DataTypeConverter converter = new DataTypeConverter(); - @Test - public void testTimeToString() { + ClickHouseColumn intColumn = ClickHouseColumn.of("v", "Array(Int32)"); + assertEquals(converter.arrayToString(new byte[]{1, 2, 3}, intColumn), "[1, 2, 3]"); + assertEquals(converter.arrayToString(new short[]{1, 2, 3}, intColumn), "[1, 2, 3]"); + assertEquals(converter.arrayToString(new int[]{1, 2, 3}, intColumn), "[1, 2, 3]"); + assertEquals(converter.arrayToString(new long[]{1L, 2L, 3L}, intColumn), "[1, 2, 3]"); + assertEquals(converter.arrayToString(new float[]{1.0f, 2.0f, 3.0f}, intColumn), "[1.0, 2.0, 3.0]"); + assertEquals(converter.arrayToString(new double[]{1.0d, 2.0d, 3.0d}, intColumn), "[1.0, 2.0, 3.0]"); + assertEquals(converter.arrayToString(new boolean[]{true, false, true}, intColumn), "[true, false, true]"); + + + ClickHouseColumn strColumn = ClickHouseColumn.of("v", "Array(String)"); + assertEquals(converter.arrayToString(new String[][]{{"a", null}, {"b", "c"}}, strColumn), "[['a', NULL], ['b', 'c']]"); + assertEquals(converter.arrayToString(new int[][]{{1, 2}, {3, 4}}, intColumn), "[[1, 2], [3, 4]]"); + assertEquals(converter.arrayToString(new int[][][]{{{1, 2}, {3, 4}}, {{5, 6}}}, intColumn), "[[[1, 2], [3, 4]], [[5, 6]]]"); + assertEquals(converter.arrayToString(new char[]{'a', 'b', 'c'}, strColumn), "['a', 'b', 'c']"); } @Test - public void testDateTimeToString() { + public void testListToString() { + DataTypeConverter converter = new DataTypeConverter(); + ClickHouseColumn column = ClickHouseColumn.of("field", "Array(Int32)"); + assertEquals(converter.arrayToString(Collections.emptyList(), column), "[]"); + assertEquals(converter.arrayToString(Arrays.asList(1, 2, 3), column), "[1, 2, 3]"); + assertEquals(converter.arrayToString(Arrays.asList(1, null, 3), column), "[1, NULL, 3]"); + assertEquals(converter.arrayToString(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4)), column), "[[1, 2], [3, 4]]"); + assertEquals(converter.arrayToString(Arrays.asList(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4)), Arrays.asList(Arrays.asList(5, 6))), column), "[[[1, 2], [3, 4]], [[5, 6]]]"); + assertEquals(converter.arrayToString(Arrays.asList(null, null, null), column), "[NULL, NULL, NULL]"); } } \ No newline at end of file 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 ddc947205..08c45118a 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 @@ -579,9 +579,9 @@ public void testDynamicWithArrays() throws Exception { }, new String[]{ "a,b", - "[a, null, b]", + "[a, NULL, b]", "[c, d]", - "[1, null, 2, null, 3]" + "[1, NULL, 2, NULL, 3]" }); testDynamicWith("arrays", new Object[]{ From 0b5dddb0d892fb79872497c1476956026326a191 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Mon, 22 Sep 2025 23:27:40 -0700 Subject: [PATCH 3/6] added more tests for string conversion --- .../clickhouse/client/api/DataTypeUtils.java | 14 +-- .../internal/MapBackedRecord.java | 1 - .../api/internal/BaseCollectionConverter.java | 5 +- .../api/internal/DataTypeConverter.java | 43 +++++-- .../api/internal/DataTypeConverterTest.java | 118 +++++++++++++++--- .../com/clickhouse/jdbc/StatementTest.java | 2 - 6 files changed, 134 insertions(+), 49 deletions(-) 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 ce62a86b4..b9f0218cc 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,9 +1,7 @@ package com.clickhouse.client.api; -import java.text.SimpleDateFormat; -import java.time.Instant; -import java.time.ZoneId; import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader; +import com.clickhouse.data.ClickHouseDataType; import java.time.Instant; import java.time.ZoneId; @@ -12,8 +10,6 @@ import java.time.temporal.ChronoField; import java.util.Objects; -import com.clickhouse.data.ClickHouseDataType; - import static com.clickhouse.client.api.data_formats.internal.BinaryStreamReader.BASES; public class DataTypeUtils { @@ -44,14 +40,6 @@ public class DataTypeUtils { public static final DateTimeFormatter TIME_FORMATTER = DateTimeFormatter.ofPattern("HH:mm:ss"); - public static final SimpleDateFormat OLD_DATE_TIME_FORMATTER = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); - - public static final SimpleDateFormat OLD_DATE_FORMATTER = new SimpleDateFormat("yyyy-MM-dd"); - - public static final SimpleDateFormat OLD_TIME_FORMATTER = new SimpleDateFormat("HH:mm:ss"); - - public static final SimpleDateFormat OLD_TIME_WITH_NANOS_FORMATTER = new SimpleDateFormat("HH:mm:ss.SSSSSSSSS"); - /** * Formats an {@link Instant} object for use in SQL statements or as query * parameter. 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 d881bca45..1e729285c 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 @@ -25,7 +25,6 @@ import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.time.temporal.TemporalAmount; -import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/BaseCollectionConverter.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/BaseCollectionConverter.java index f051b8f34..1c058f0cd 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/BaseCollectionConverter.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/BaseCollectionConverter.java @@ -4,8 +4,9 @@ import java.io.IOException; import java.lang.reflect.Array; +import java.util.ArrayDeque; +import java.util.Deque; import java.util.List; -import java.util.Stack; public abstract class BaseCollectionConverter { public static final String ARRAY_START = "["; @@ -45,7 +46,7 @@ public final String convert(TList value, TAcc acc) { } setAccumulator(acc); - Stack> stack = new Stack<>(); + Deque> stack = new ArrayDeque<>(); ListConversionState state = new ListConversionState<>(value, listSize(value)); while (state != null) { if (state.isFirst()) { diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java index 3f903c829..64b1b0b33 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java @@ -7,9 +7,11 @@ import com.clickhouse.data.ClickHouseDataType; import java.net.InetAddress; +import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.time.ZoneId; import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; import java.time.temporal.TemporalAccessor; @@ -76,7 +78,7 @@ public String stringToString(Object bytesOrString, ClickHouseColumn column) { if (bytesOrString instanceof CharSequence) { sb.append(((CharSequence) bytesOrString)); } else if (bytesOrString instanceof byte[]) { - sb.append(bytesOrString); + sb.append(new String((byte[]) bytesOrString)); } else { sb.append(bytesOrString); } @@ -86,6 +88,11 @@ public String stringToString(Object bytesOrString, ClickHouseColumn column) { return sb.toString(); } + public static ZoneId UTC_ZONE_ID = ZoneId.of("UTC"); + + public static final ZonedDateTime EPOCH_START_OF_THE_DAY = + ZonedDateTime.ofInstant(Instant.EPOCH, UTC_ZONE_ID); + public String dateToString(Object value, ClickHouseColumn column) { DateTimeFormatter formatter = DataTypeUtils.DATE_FORMATTER; @@ -94,12 +101,19 @@ public String dateToString(Object value, ClickHouseColumn column) { return formatter.format(dateTime); } else if (value instanceof LocalDate) { return formatter.format(((LocalDate)value)); + } else if (value instanceof java.sql.Date) { + java.sql.Date date = (java.sql.Date) value; + return formatter.format(ZonedDateTime.ofInstant(Instant.ofEpochMilli(date.getTime()), UTC_ZONE_ID)); + } else if (value instanceof java.sql.Time) { + return formatter.format(EPOCH_START_OF_THE_DAY); } else if (value instanceof Date) { - return DataTypeUtils.OLD_DATE_FORMATTER.format(((Date)value)); + return formatter.format(((Date)value).toInstant().atZone(UTC_ZONE_ID)); } return value.toString(); } + + public String timeToString(Object value, ClickHouseColumn column) { DateTimeFormatter formatter; switch (column.getDataType()) { @@ -115,8 +129,13 @@ public String timeToString(Object value, ClickHouseColumn column) { return formatter.format(dateTime); } else if (value instanceof LocalTime) { return formatter.format(((LocalTime)value)); + } else if (value instanceof java.sql.Date) { + return formatter.format(EPOCH_START_OF_THE_DAY); + } else if (value instanceof java.sql.Time) { + java.sql.Time date = (java.sql.Time) value; + return formatter.format(ZonedDateTime.ofInstant(Instant.ofEpochMilli(date.getTime()), UTC_ZONE_ID)); } else if (value instanceof Date) { - return DataTypeUtils.OLD_TIME_FORMATTER.format(((Date)value)); + return formatter.format(((Date)value).toInstant().atZone(UTC_ZONE_ID)); } return value.toString(); } @@ -137,9 +156,15 @@ public String dateTimeToString(Object value, ClickHouseColumn column) { } else if (value instanceof LocalDate) { return formatter.format(((LocalDate)value).atStartOfDay()); } else if (value instanceof LocalTime) { - return formatter.format(((LocalTime)value).atDate(LocalDate.now())); + return formatter.format(((LocalTime) value).atDate(LocalDate.now())); + } else if (value instanceof java.sql.Date) { + return formatter.format(EPOCH_START_OF_THE_DAY); + + } else if (value instanceof java.sql.Time) { + java.sql.Time date = (java.sql.Time) value; + return formatter.format(ZonedDateTime.ofInstant(Instant.ofEpochMilli(date.getTime()), UTC_ZONE_ID)); } else if (value instanceof Date) { - return DataTypeUtils.OLD_DATE_TIME_FORMATTER.format(((Date)value)); + return formatter.format(((Date)value).toInstant().atZone(UTC_ZONE_ID)); } return value.toString(); } @@ -150,14 +175,6 @@ public String enumToString(Object value, ClickHouseColumn column) { } else if (value instanceof Number ) { int num = ((Number) value).intValue(); switch (column.getDataType()) { - case Variant: - for (ClickHouseColumn c : column.getNestedColumns()) { - // TODO: will work only if single enum listed as variant - if (c.getDataType() == ClickHouseDataType.Enum8 || c.getDataType() == ClickHouseDataType.Enum16) { - return c.getEnumConstants().name(num); - } - } - return String.valueOf(num); // fail-safe case Enum8: case Enum16: case Enum: diff --git a/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java b/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java index 87759882f..4c7778785 100644 --- a/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java +++ b/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java @@ -1,13 +1,19 @@ package com.clickhouse.client.api.internal; import com.clickhouse.data.ClickHouseColumn; -import com.clickhouse.data.ClickHouseDataType; import org.testng.annotations.Test; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.ZonedDateTime; import java.util.Arrays; import java.util.Collections; +import java.util.Date; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; @Test(groups = {"unit"}) public class DataTypeConverterTest { @@ -17,31 +23,107 @@ public void testArrayToString() { DataTypeConverter converter = new DataTypeConverter(); ClickHouseColumn intColumn = ClickHouseColumn.of("v", "Array(Int32)"); - assertEquals(converter.arrayToString(new byte[]{1, 2, 3}, intColumn), "[1, 2, 3]"); - assertEquals(converter.arrayToString(new short[]{1, 2, 3}, intColumn), "[1, 2, 3]"); - assertEquals(converter.arrayToString(new int[]{1, 2, 3}, intColumn), "[1, 2, 3]"); - assertEquals(converter.arrayToString(new long[]{1L, 2L, 3L}, intColumn), "[1, 2, 3]"); - assertEquals(converter.arrayToString(new float[]{1.0f, 2.0f, 3.0f}, intColumn), "[1.0, 2.0, 3.0]"); - assertEquals(converter.arrayToString(new double[]{1.0d, 2.0d, 3.0d}, intColumn), "[1.0, 2.0, 3.0]"); - assertEquals(converter.arrayToString(new boolean[]{true, false, true}, intColumn), "[true, false, true]"); + assertEquals(converter.convertToString(new byte[]{1, 2, 3}, intColumn), "[1, 2, 3]"); + assertEquals(converter.convertToString(new short[]{1, 2, 3}, intColumn), "[1, 2, 3]"); + assertEquals(converter.convertToString(new int[]{1, 2, 3}, intColumn), "[1, 2, 3]"); + assertEquals(converter.convertToString(new long[]{1L, 2L, 3L}, intColumn), "[1, 2, 3]"); + assertEquals(converter.convertToString(new float[]{1.0f, 2.0f, 3.0f}, intColumn), "[1.0, 2.0, 3.0]"); + assertEquals(converter.convertToString(new double[]{1.0d, 2.0d, 3.0d}, intColumn), "[1.0, 2.0, 3.0]"); + assertEquals(converter.convertToString(new boolean[]{true, false, true}, intColumn), "[true, false, true]"); ClickHouseColumn strColumn = ClickHouseColumn.of("v", "Array(String)"); - assertEquals(converter.arrayToString(new String[][]{{"a", null}, {"b", "c"}}, strColumn), "[['a', NULL], ['b', 'c']]"); - assertEquals(converter.arrayToString(new int[][]{{1, 2}, {3, 4}}, intColumn), "[[1, 2], [3, 4]]"); - assertEquals(converter.arrayToString(new int[][][]{{{1, 2}, {3, 4}}, {{5, 6}}}, intColumn), "[[[1, 2], [3, 4]], [[5, 6]]]"); - assertEquals(converter.arrayToString(new char[]{'a', 'b', 'c'}, strColumn), "['a', 'b', 'c']"); + assertEquals(converter.convertToString(new String[][]{{"a", null}, {"b", "c"}}, strColumn), "[['a', NULL], ['b', 'c']]"); + assertEquals(converter.convertToString(new int[][]{{1, 2}, {3, 4}}, intColumn), "[[1, 2], [3, 4]]"); + assertEquals(converter.convertToString(new int[][][]{{{1, 2}, {3, 4}}, {{5, 6}}}, intColumn), "[[[1, 2], [3, 4]], [[5, 6]]]"); + assertEquals(converter.convertToString(new char[]{'a', 'b', 'c'}, strColumn), "['a', 'b', 'c']"); } @Test public void testListToString() { DataTypeConverter converter = new DataTypeConverter(); ClickHouseColumn column = ClickHouseColumn.of("field", "Array(Int32)"); - assertEquals(converter.arrayToString(Collections.emptyList(), column), "[]"); - assertEquals(converter.arrayToString(Arrays.asList(1, 2, 3), column), "[1, 2, 3]"); - assertEquals(converter.arrayToString(Arrays.asList(1, null, 3), column), "[1, NULL, 3]"); - assertEquals(converter.arrayToString(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4)), column), "[[1, 2], [3, 4]]"); - assertEquals(converter.arrayToString(Arrays.asList(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4)), Arrays.asList(Arrays.asList(5, 6))), column), "[[[1, 2], [3, 4]], [[5, 6]]]"); - assertEquals(converter.arrayToString(Arrays.asList(null, null, null), column), "[NULL, NULL, NULL]"); + assertEquals(converter.convertToString(Collections.emptyList(), column), "[]"); + assertEquals(converter.convertToString(Arrays.asList(1, 2, 3), column), "[1, 2, 3]"); + assertEquals(converter.convertToString(Arrays.asList(1, null, 3), column), "[1, NULL, 3]"); + assertEquals(converter.convertToString(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4)), column), "[[1, 2], [3, 4]]"); + assertEquals(converter.convertToString(Arrays.asList(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4)), Arrays.asList(Arrays.asList(5, 6))), column), "[[[1, 2], [3, 4]], [[5, 6]]]"); + assertEquals(converter.convertToString(Arrays.asList(null, null, null), column), "[NULL, NULL, NULL]"); + } + + @Test + public void testDateToString() { + DataTypeConverter converter = new DataTypeConverter(); + ClickHouseColumn column = ClickHouseColumn.of("field", "Date"); + assertEquals(converter.convertToString(LocalDate.of(2022, 1, 1), column), "2022-01-01"); + assertEquals(converter.convertToString(LocalDate.of(2022, 1, 2), column), "2022-01-02"); + assertEquals(converter.convertToString(LocalDate.of(2022, 1, 3), column), "2022-01-03"); + + Date date = Date.from(ZonedDateTime.of(2022, 1, 4, 12, 34, 56, 0, ZoneId.of("Asia/Shanghai")).toInstant()); + assertEquals(converter.convertToString(date, column), "2022-01-04"); + Date sqlDate = java.sql.Date.valueOf("2022-01-04"); + assertEquals(converter.convertToString(sqlDate, column), "2022-01-04"); + java.sql.Time sqlTime = java.sql.Time.valueOf("12:34:56"); + assertEquals(converter.convertToString(sqlTime, column), "1970-01-01"); + } + + + @Test + public void testTimeToString() { + DataTypeConverter converter = new DataTypeConverter(); + ClickHouseColumn column = ClickHouseColumn.of("field", "Time"); + assertEquals(converter.timeToString(LocalTime.of(12, 34, 56), column), "12:34:56"); + assertEquals(converter.timeToString(LocalTime.of(23, 59, 59), column), "23:59:59"); + Date sqlDate = java.sql.Date.valueOf("2022-01-04"); + assertEquals(converter.convertToString(sqlDate, column), "00:00:00"); + Date date = Date.from(ZonedDateTime.of(2022, 1, 4, 12, 34, 56, 0, ZoneId.of("Asia/Shanghai")).toInstant()); + assertEquals(converter.convertToString(date, column), "04:34:56"); + java.sql.Time sqlTime = java.sql.Time.valueOf("12:34:56"); + assertEquals(converter.convertToString(sqlTime, column), "20:34:56"); + } + + + @Test + public void testDateTimeToString() { + DataTypeConverter converter = new DataTypeConverter(); + ClickHouseColumn column = ClickHouseColumn.of("field", "DateTime"); + assertEquals(converter.dateTimeToString(LocalDateTime.of(2022, 1, 1, 12, 34, 56), column), "2022-01-01 12:34:56"); + assertEquals(converter.dateTimeToString(LocalDateTime.of(2022, 1, 2, 23, 59, 59), column), "2022-01-02 23:59:59"); + + Date date = Date.from(ZonedDateTime.of(2022, 1, 4, 12, 34, 56, 0, ZoneId.of("Asia/Shanghai")).toInstant()); + assertEquals(converter.convertToString(date, column), "2022-01-04 04:34:56"); + Date sqlDate = java.sql.Date.valueOf("2022-01-04"); + assertEquals(converter.convertToString(sqlDate, column), "1970-01-01 00:00:00"); + java.sql.Time sqlTime = java.sql.Time.valueOf("12:34:56"); + assertEquals(converter.convertToString(sqlTime, column), "1970-01-01 20:34:56"); + } + + @Test + public void testEnumToString() { + DataTypeConverter converter = new DataTypeConverter(); + ClickHouseColumn column = ClickHouseColumn.of("field", "Enum8('a' = 1, 'b' = 2)"); + assertEquals(converter.convertToString("a", column), "a"); + assertNull(converter.convertToString(null, column)); + assertEquals(converter.convertToString(1, column), "a"); + assertEquals(converter.convertToString("1234567", column), "1234567"); + + column = ClickHouseColumn.of("field", "Enum8('a' = 1, 'b' = 2)"); + assertEquals(converter.convertToString("a", column), "a"); + assertNull(converter.convertToString(null, column)); + assertEquals(converter.convertToString(1, column), "a"); + assertEquals(converter.convertToString("1234567", column), "1234567"); + + column = ClickHouseColumn.of("field", "Enum8('a' = 1, 'b' = 2)"); + assertEquals(converter.convertToString("a", column), "a"); + assertNull(converter.convertToString(null, column)); + assertEquals(converter.convertToString(2, column), "b"); + assertEquals(converter.convertToString("1234567", column), "1234567"); + + column = ClickHouseColumn.of("field", "Variant(Enum8('a' = 1, 'b' = 2))"); + assertEquals(converter.convertToString("a", column), "a"); + assertEquals(converter.convertToString(null, column), null); + assertEquals(converter.convertToString(1, column), "1"); + assertEquals(converter.convertToString("1234567", column), "1234567"); + assertEquals(converter.convertToString(2, column), "2"); } } \ No newline at end of file diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java index a78d848ff..46ec48405 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java @@ -11,8 +11,6 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.net.Inet4Address; -import java.net.Inet6Address; import java.sql.Array; import java.sql.Connection; import java.sql.Date; From e78db3820d13c56971ed267ff9675bbb987d735b Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 23 Sep 2025 00:07:49 -0700 Subject: [PATCH 4/6] fixed converter for dynamic types and make tool to enquote lists --- .../api/internal/DataTypeConverter.java | 36 +++++++++++++------ .../clickhouse/client/query/QueryTests.java | 15 ++++++++ 2 files changed, 40 insertions(+), 11 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java index 64b1b0b33..74c589a2c 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java @@ -6,6 +6,7 @@ import com.clickhouse.data.ClickHouseColumn; import com.clickhouse.data.ClickHouseDataType; +import java.io.IOException; import java.net.InetAddress; import java.time.Instant; import java.time.LocalDate; @@ -202,6 +203,11 @@ public String arrayToString(Object value, ClickHouseColumn column) { return value.toString(); } + public String arrayToString(Object value, String columnDef) { + ClickHouseColumn column = ClickHouseColumn.of("v", columnDef); + return arrayToString(value, column); + } + /** * * @param value not null object value to convert @@ -215,6 +221,18 @@ public String variantOrDynamicToString(Object value, ClickHouseColumn column) { return value.toString(); } + private static void appendEnquotedArrayElement(String value, ClickHouseColumn elementColumn, Appendable appendable) { + try { + if (elementColumn != null && elementColumn.getDataType() == ClickHouseDataType.String) { + appendable.append(QUOTE).append(value).append(QUOTE); + } else { + appendable.append(value); + } + } catch (IOException e) { + throw new ClickHouseException(e.getMessage(), e); + } + } + private final class ArrayAsStringWriter extends BaseCollectionConverter.BaseArrayWriter { private ClickHouseColumn column; @@ -233,16 +251,9 @@ protected void onItem(Object item, ListConversionState state) { append(NULL); return; } - String str = DataTypeConverter.this.convertToString(item, column.getArrayBaseColumn() == null ? column : column.getArrayBaseColumn()); - try { - if (column.getArrayBaseColumn().getDataType() == ClickHouseDataType.String) { - appendable.append(QUOTE).append(str).append(QUOTE); - } else { - appendable.append(str); - } - } catch (Exception ex) { - throw new ClickHouseException(ex.getMessage(), ex); - } + ClickHouseColumn elementColumn = column.getArrayBaseColumn() == null ? column : column.getArrayBaseColumn(); + String str = DataTypeConverter.this.convertToString(item, elementColumn); + appendEnquotedArrayElement(str, elementColumn, appendable); } public String convertAndReset(Object list, Appendable acc, ClickHouseColumn column) { @@ -270,7 +281,10 @@ protected void onItem(Object item, ListConversionState> state) { append(NULL); return; } - append(DataTypeConverter.this.convertToString(item, column.getArrayBaseColumn() == null ? column : column.getArrayBaseColumn())); + ClickHouseColumn elementColumn = column.getArrayBaseColumn() == null ? column : column.getArrayBaseColumn(); + String str = DataTypeConverter.this.convertToString(item, elementColumn); + appendEnquotedArrayElement(str, elementColumn, appendable); + } public String convertAndReset(List list, Appendable acc, ClickHouseColumn column) { 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 a35746143..d3dd90eae 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 @@ -16,6 +16,7 @@ import com.clickhouse.client.api.enums.Protocol; import com.clickhouse.client.api.insert.InsertResponse; import com.clickhouse.client.api.insert.InsertSettings; +import com.clickhouse.client.api.internal.DataTypeConverter; import com.clickhouse.client.api.internal.ServerSettings; import com.clickhouse.client.api.internal.StopWatch; import com.clickhouse.client.api.metadata.TableSchema; @@ -1634,6 +1635,20 @@ public void testQueryParams() throws Exception { Assert.assertEquals(allRecords.size(), 2); } + @Test(groups = {"integration"}) + public void testQueryParamsWithArrays() { + final Map params = new HashMap<>(); + params.put("database_name", "system"); + params.put("table_names", + DataTypeConverter.INSTANCE.arrayToString(Arrays.asList("COLLATIONS", "ENGINES"), "Array(String)")); + // This query should not throw an exception + List records = client.queryAll("SELECT database, name FROM system.tables WHERE name IN {table_names:Array(String)}", + params); + + Assert.assertEquals(records.get(0).getString("name"), "COLLATIONS"); + Assert.assertEquals(records.get(1).getString("name"), "ENGINES"); + } + @Test(groups = {"integration"}) public void testExecuteQueryParam() throws ExecutionException, InterruptedException, TimeoutException { From e1cd563ea84c8f051cf5441ffe45b30ed85b9fa9 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 23 Sep 2025 00:38:28 -0700 Subject: [PATCH 5/6] fix sql.Time? --- .../client/api/internal/DataTypeConverter.java | 12 ++++++++---- .../client/api/internal/DataTypeConverterTest.java | 6 +++--- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java index 74c589a2c..883ded278 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/DataTypeConverter.java @@ -94,6 +94,8 @@ public String stringToString(Object bytesOrString, ClickHouseColumn column) { public static final ZonedDateTime EPOCH_START_OF_THE_DAY = ZonedDateTime.ofInstant(Instant.EPOCH, UTC_ZONE_ID); + public static final LocalDate EPOCH_DATE = LocalDate.of(1970, 1, 1); + public String dateToString(Object value, ClickHouseColumn column) { DateTimeFormatter formatter = DataTypeUtils.DATE_FORMATTER; @@ -133,8 +135,9 @@ public String timeToString(Object value, ClickHouseColumn column) { } else if (value instanceof java.sql.Date) { return formatter.format(EPOCH_START_OF_THE_DAY); } else if (value instanceof java.sql.Time) { - java.sql.Time date = (java.sql.Time) value; - return formatter.format(ZonedDateTime.ofInstant(Instant.ofEpochMilli(date.getTime()), UTC_ZONE_ID)); + java.sql.Time time = (java.sql.Time) value; + LocalTime lt = time.toLocalTime(); + return formatter.format(lt); } else if (value instanceof Date) { return formatter.format(((Date)value).toInstant().atZone(UTC_ZONE_ID)); } @@ -162,8 +165,9 @@ public String dateTimeToString(Object value, ClickHouseColumn column) { return formatter.format(EPOCH_START_OF_THE_DAY); } else if (value instanceof java.sql.Time) { - java.sql.Time date = (java.sql.Time) value; - return formatter.format(ZonedDateTime.ofInstant(Instant.ofEpochMilli(date.getTime()), UTC_ZONE_ID)); + java.sql.Time time = (java.sql.Time) value; + LocalTime lt = time.toLocalTime(); + return formatter.format(lt.atDate(EPOCH_DATE)); } else if (value instanceof Date) { return formatter.format(((Date)value).toInstant().atZone(UTC_ZONE_ID)); } diff --git a/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java b/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java index 4c7778785..254f44fdd 100644 --- a/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java +++ b/client-v2/src/test/java/com/clickhouse/client/api/internal/DataTypeConverterTest.java @@ -78,8 +78,8 @@ public void testTimeToString() { assertEquals(converter.convertToString(sqlDate, column), "00:00:00"); Date date = Date.from(ZonedDateTime.of(2022, 1, 4, 12, 34, 56, 0, ZoneId.of("Asia/Shanghai")).toInstant()); assertEquals(converter.convertToString(date, column), "04:34:56"); - java.sql.Time sqlTime = java.sql.Time.valueOf("12:34:56"); - assertEquals(converter.convertToString(sqlTime, column), "20:34:56"); + java.sql.Time sqlTime = java.sql.Time.valueOf("12:34:56");// 12:34:56 in ms + assertEquals(converter.convertToString(sqlTime, column), "12:34:56"); } @@ -95,7 +95,7 @@ public void testDateTimeToString() { Date sqlDate = java.sql.Date.valueOf("2022-01-04"); assertEquals(converter.convertToString(sqlDate, column), "1970-01-01 00:00:00"); java.sql.Time sqlTime = java.sql.Time.valueOf("12:34:56"); - assertEquals(converter.convertToString(sqlTime, column), "1970-01-01 20:34:56"); + assertEquals(converter.convertToString(sqlTime, column), "1970-01-01 12:34:56"); } @Test From 9cb45a5508b60f33f117f62d3749e40950b3d338 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 23 Sep 2025 01:03:17 -0700 Subject: [PATCH 6/6] fixed minor issues --- .../api/internal/BaseCollectionConverter.java | 40 +++++++++---------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/BaseCollectionConverter.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/BaseCollectionConverter.java index 1c058f0cd..7ea080bee 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/BaseCollectionConverter.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/BaseCollectionConverter.java @@ -8,7 +8,7 @@ import java.util.Deque; import java.util.List; -public abstract class BaseCollectionConverter { +public abstract class BaseCollectionConverter { public static final String ARRAY_START = "["; public static final String ARRAY_END = "]"; @@ -18,36 +18,36 @@ protected BaseCollectionConverter(String itemDelimiter) { this.itemDelimiter = itemDelimiter; } - protected abstract void setAccumulator(TAcc acc); + protected abstract void setAccumulator(ACC acc); protected abstract void append(String str); protected abstract String buildString(); - protected abstract void onStart(ListConversionState state); + protected abstract void onStart(ListConversionState state); - protected abstract void onEnd(ListConversionState state); + protected abstract void onEnd(ListConversionState state); - protected abstract void onItem(Object item, ListConversionState state); + protected abstract void onItem(Object item, ListConversionState state); protected abstract String onEmptyCollection(); - protected abstract boolean isEmpty(TList list); + protected abstract boolean isEmpty(LIST list); protected abstract boolean isSubList(Object list); - protected abstract int listSize(TList list); + protected abstract int listSize(LIST list); - protected abstract Object getNext(ListConversionState state); + protected abstract Object getNext(ListConversionState state); - public final String convert(TList value, TAcc acc) { + public final String convert(LIST value, ACC acc) { if (isEmpty(value)) { return onEmptyCollection(); } setAccumulator(acc); - Deque> stack = new ArrayDeque<>(); - ListConversionState state = new ListConversionState<>(value, listSize(value)); + Deque> stack = new ArrayDeque<>(); + ListConversionState state = new ListConversionState<>(value, listSize(value)); while (state != null) { if (state.isFirst()) { onStart(state); @@ -57,7 +57,7 @@ public final String convert(TList value, TAcc acc) { state.incPos(); if (isSubList(item)) { stack.push(state); - TList list = (TList) item; + LIST list = (LIST) item; state = new ListConversionState<>(list, listSize(list)); } else { onItem(item, state); @@ -77,19 +77,19 @@ public final String convert(TList value, TAcc acc) { return buildString(); } - public static final class ListConversionState { + public static final class ListConversionState { - final TList list; + final LIST list; int position; int size; - public ListConversionState(TList list, int size) { + private ListConversionState(LIST list, int size) { this.list = list; this.position = 0; this.size = size; } - public TList getList() { + public LIST getList() { return list; } @@ -110,7 +110,7 @@ public boolean isFirst() { } } - public static abstract class BaseArrayWriter extends BaseCollectionWriter { + public abstract static class BaseArrayWriter extends BaseCollectionWriter { protected BaseArrayWriter() { super(", "); @@ -137,9 +137,9 @@ protected Object getNext(ListConversionState state) { } } - public static abstract class BaseListWriter + public abstract static class BaseListWriter extends BaseCollectionWriter> { - public BaseListWriter() { + protected BaseListWriter() { super(", "); } @@ -164,7 +164,7 @@ protected Object getNext(ListConversionState> state) { } } - public static abstract class BaseCollectionWriter extends + public abstract static class BaseCollectionWriter extends BaseCollectionConverter { protected Appendable appendable;