From aa857ed83fa5bcb2d203dd7ef61c401a77f1d33e Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Fri, 27 Jun 2025 14:04:46 -0700 Subject: [PATCH 1/6] Implemeted configuration for default type hint --- .../com/clickhouse/client/api/Client.java | 50 +++++++++++-- .../client/api/ClientConfigProperties.java | 74 +++++++++++++++++++ .../api/data_formats/NativeFormatReader.java | 11 ++- .../data_formats/RowBinaryFormatReader.java | 17 ++++- ...owBinaryWithNamesAndTypesFormatReader.java | 16 +++- .../RowBinaryWithNamesFormatReader.java | 18 ++++- .../internal/AbstractBinaryFormatReader.java | 8 +- .../internal/BinaryStreamReader.java | 12 ++- .../jdbc/PreparedStatementImpl.java | 26 +++++++ .../com/clickhouse/jdbc/ResultSetImpl.java | 1 + .../jdbc/internal/JdbcConfiguration.java | 10 ++- .../com/clickhouse/jdbc/DataTypeTests.java | 57 ++++++++++++++ 12 files changed, 277 insertions(+), 23 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java index cc7890fd2..b258d6b7f 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java @@ -7,6 +7,7 @@ import com.clickhouse.client.api.data_formats.RowBinaryFormatReader; import com.clickhouse.client.api.data_formats.RowBinaryWithNamesAndTypesFormatReader; import com.clickhouse.client.api.data_formats.RowBinaryWithNamesFormatReader; +import com.clickhouse.client.api.data_formats.internal.AbstractBinaryFormatReader; import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader; import com.clickhouse.client.api.data_formats.internal.MapBackedRecord; import com.clickhouse.client.api.data_formats.internal.ProcessParser; @@ -36,7 +37,9 @@ import com.clickhouse.client.api.transport.Endpoint; import com.clickhouse.client.api.transport.HttpEndpoint; import com.clickhouse.client.config.ClickHouseClientOption; +import com.clickhouse.config.ClickHouseOption; import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.ClickHouseDataType; import com.clickhouse.data.ClickHouseFormat; import com.google.common.collect.ImmutableList; import net.jpountz.lz4.LZ4Factory; @@ -131,6 +134,8 @@ public class Client implements AutoCloseable { private final Map tableSchemaHasDefaults = new ConcurrentHashMap<>(); + private final Map> typeHintMapping; + // Server context private String serverVersion; private Object metricsRegistry; @@ -191,6 +196,26 @@ private Client(Set endpoints, Map configuration, } this.serverVersion = configuration.getOrDefault(ClientConfigProperties.SERVER_VERSION.getKey(), "unknown"); + + this.typeHintMapping = translateTypeHintMapping(configuration.get(ClientConfigProperties.TYPE_HINT_MAPPING.getKey())); + } + + private Map> translateTypeHintMapping(String mappingStr) { + if (mappingStr == null || mappingStr.isEmpty()) { + return AbstractBinaryFormatReader.NO_TYPE_HINT_MAPPING; + } + + Map mapping= ClientConfigProperties.toKeyValuePairs(mappingStr); + Map> hintMapping = new HashMap<>(); + try { + for (Map.Entry entry : mapping.entrySet()) { + hintMapping.put(ClickHouseDataType.of(entry.getKey()), + Class.forName(entry.getValue())); + } + } catch (ClassNotFoundException e) { + throw new ClientMisconfigurationException("Failed to translate type-hint mapping", e); + } + return hintMapping; } /** @@ -1003,6 +1028,20 @@ public Builder setServerVersion(String serverVersion) { return this; } + /** + * Defines mapping between ClickHouse data type and target Java type + * Used by binary readers to convert values into desired Java type. + * @param typeHintMapping - map between ClickHouse data type and Java class + * @return this builder instance + */ + public Builder typeHintMapping(Map> typeHintMapping) { + this.configuration.put(ClientConfigProperties.TYPE_HINT_MAPPING.getKey(), + ClientConfigProperties.mapToString(typeHintMapping, (v) -> { + return ((Class) v).getName(); + })); + return this; + } + public Client build() { // check if endpoint are empty. so can not initiate client if (this.endpoints.isEmpty()) { @@ -1921,23 +1960,20 @@ public ClickHouseBinaryFormatReader newBinaryFormatReader(QueryResponse response BinaryStreamReader.ByteBufferAllocator byteBufferPool = useCachingBufferAllocator ? new BinaryStreamReader.CachingByteBufferAllocator() : new BinaryStreamReader.DefaultByteBufferAllocator(); - switch (response.getFormat()) { case Native: reader = new NativeFormatReader(response.getInputStream(), response.getSettings(), - byteBufferPool); + byteBufferPool, typeHintMapping); break; case RowBinaryWithNamesAndTypes: - reader = new RowBinaryWithNamesAndTypesFormatReader(response.getInputStream(), response.getSettings(), - byteBufferPool); + reader = new RowBinaryWithNamesAndTypesFormatReader(response.getInputStream(), response.getSettings(), byteBufferPool, typeHintMapping); break; case RowBinaryWithNames: - reader = new RowBinaryWithNamesFormatReader(response.getInputStream(), response.getSettings(), schema, - byteBufferPool); + reader = new RowBinaryWithNamesFormatReader(response.getInputStream(), response.getSettings(), schema, byteBufferPool, typeHintMapping); break; case RowBinary: reader = new RowBinaryFormatReader(response.getInputStream(), response.getSettings(), schema, - byteBufferPool); + byteBufferPool, typeHintMapping); break; default: throw new IllegalArgumentException("Binary readers doesn't support format: " + response.getFormat()); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java b/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java index 6b1626ed0..f385162fc 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java @@ -5,8 +5,11 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; import java.util.Locale; +import java.util.Map; +import java.util.function.Function; import java.util.stream.Collectors; /** @@ -140,6 +143,12 @@ public enum ClientConfigProperties { * Name of the group under which client metrics appear */ METRICS_GROUP_NAME("metrics_name"), + + /** + * Defines mapping between ClickHouse data type and target Java type + * Used by binary readers to convert values into desired Java type. + */ + TYPE_HINT_MAPPING("type_hint_mapping"), ; private final String key; @@ -207,4 +216,69 @@ public static List valuesFromCommaSeparated(String value) { return Arrays.stream(value.split("(? s.replaceAll("\\\\,", ",")) .collect(Collectors.toList()); } + + + /** + * Converts given string to key value pairs. + * + * @param str string + * @return non-null key value pairs + */ + public static Map toKeyValuePairs(String str) { + if (str == null || str.isEmpty()) { + return Collections.emptyMap(); + } + + Map map = new LinkedHashMap<>(); + String key = null; + StringBuilder builder = new StringBuilder(); + for (int i = 0, len = str.length(); i < len; i++) { + char ch = str.charAt(i); + if (ch == '\\' && i + 1 < len) { + ch = str.charAt(++i); + builder.append(ch); + continue; + } + + if (Character.isWhitespace(ch)) { + if (builder.length() > 0) { + builder.append(ch); + } + } else if (ch == '=' && key == null) { + key = builder.toString().trim(); + builder.setLength(0); + } else if (ch == ',' && key != null) { + String value = builder.toString().trim(); + builder.setLength(0); + if (!key.isEmpty() && !value.isEmpty()) { + map.put(key, value); + } + key = null; + } else { + builder.append(ch); + } + } + + if (key != null && builder.length() > 0) { + String value = builder.toString().trim(); + if (!key.isEmpty() && !value.isEmpty()) { + map.put(key, value); + } + } + + return Collections.unmodifiableMap(map); + } + + + public static String mapToString(Map map, Function valueConverter) { + StringBuilder sb = new StringBuilder(); + for (Map.Entry entry : map.entrySet()) { + sb.append(entry.getKey()).append("=").append(valueConverter.apply(entry.getValue())).append(","); + } + + if (sb.length() > 0) { + sb.setLength(sb.length() - 1); + } + return sb.toString(); + } } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/NativeFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/NativeFormatReader.java index e559080f9..ddcf0049b 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/NativeFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/NativeFormatReader.java @@ -6,6 +6,7 @@ import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.QuerySettings; import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.ClickHouseDataType; import java.io.EOFException; import java.io.IOException; @@ -26,8 +27,9 @@ public class NativeFormatReader extends AbstractBinaryFormatReader { private int blockRowIndex; public NativeFormatReader(InputStream inputStream, QuerySettings settings, - BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) { - super(inputStream, settings, null, byteBufferAllocator); + BinaryStreamReader.ByteBufferAllocator byteBufferAllocator, + Map> typeHintMapping) { + super(inputStream, settings, null, byteBufferAllocator, typeHintMapping); try { readBlock(); } catch (IOException e) { @@ -35,6 +37,11 @@ public NativeFormatReader(InputStream inputStream, QuerySettings settings, } } + public NativeFormatReader(InputStream inputStream, QuerySettings settings, + BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) { + this(inputStream, settings, byteBufferAllocator, NO_TYPE_HINT_MAPPING); + } + @Override public boolean readRecord(Map record) throws IOException { if (blockRowIndex >= currentBlock.getnRows()) { diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryFormatReader.java index 35a39803c..0d8575bb9 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryFormatReader.java @@ -5,6 +5,7 @@ import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.QuerySettings; import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.ClickHouseDataType; import java.io.EOFException; import java.io.IOException; @@ -13,12 +14,22 @@ public class RowBinaryFormatReader extends AbstractBinaryFormatReader { - public RowBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema, - BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) { - super(inputStream, querySettings, schema, byteBufferAllocator); + public RowBinaryFormatReader(InputStream inputStream, + QuerySettings querySettings, + TableSchema schema, + BinaryStreamReader.ByteBufferAllocator byteBufferAllocator, + Map> typeHintMapping) { + super(inputStream, querySettings, schema, byteBufferAllocator, typeHintMapping); readNextRecord(); } + public RowBinaryFormatReader(InputStream inputStream, + QuerySettings querySettings, + TableSchema schema, + BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) { + this(inputStream, querySettings, schema, byteBufferAllocator, NO_TYPE_HINT_MAPPING); + } + @Override public boolean readRecord(Map record) throws IOException { boolean firstColumn = true; diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryWithNamesAndTypesFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryWithNamesAndTypesFormatReader.java index 7eaa13d06..67bda8b84 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryWithNamesAndTypesFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryWithNamesAndTypesFormatReader.java @@ -6,6 +6,7 @@ import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.QuerySettings; import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.ClickHouseDataType; import java.io.EOFException; import java.io.IOException; @@ -17,12 +18,21 @@ public class RowBinaryWithNamesAndTypesFormatReader extends AbstractBinaryFormatReader implements Iterator> { - public RowBinaryWithNamesAndTypesFormatReader(InputStream inputStream, QuerySettings querySettings, - BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) { - super(inputStream, querySettings, null, byteBufferAllocator); + public RowBinaryWithNamesAndTypesFormatReader(InputStream inputStream, + QuerySettings querySettings, + BinaryStreamReader.ByteBufferAllocator byteBufferAllocator, + Map> typeHintMapping) { + super(inputStream, querySettings, null, byteBufferAllocator, typeHintMapping); readSchema(); } + public RowBinaryWithNamesAndTypesFormatReader(InputStream inputStream, + QuerySettings querySettings, + BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) { + this(inputStream, querySettings, byteBufferAllocator, NO_TYPE_HINT_MAPPING); + } + + private void readSchema() { try { List names = new ArrayList<>(); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryWithNamesFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryWithNamesFormatReader.java index 3fe8769ee..fcfaa7625 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryWithNamesFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryWithNamesFormatReader.java @@ -4,6 +4,7 @@ import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader; import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.QuerySettings; +import com.clickhouse.data.ClickHouseDataType; import java.io.EOFException; import java.io.IOException; @@ -11,14 +12,18 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; public class RowBinaryWithNamesFormatReader extends AbstractBinaryFormatReader { private List columns = null; - public RowBinaryWithNamesFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema, - BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) { - super(inputStream, querySettings, schema, byteBufferAllocator); + public RowBinaryWithNamesFormatReader(InputStream inputStream, + QuerySettings querySettings, + TableSchema schema, + BinaryStreamReader.ByteBufferAllocator byteBufferAllocator, + Map> typeHintMapping) { + super(inputStream, querySettings, schema, byteBufferAllocator, typeHintMapping); int nCol = 0; try { nCol = BinaryStreamReader.readVarInt(input); @@ -44,6 +49,13 @@ public RowBinaryWithNamesFormatReader(InputStream inputStream, QuerySettings que readNextRecord(); } + public RowBinaryWithNamesFormatReader(InputStream inputStream, + QuerySettings querySettings, + TableSchema schema, + BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) { + this(inputStream, querySettings, schema, byteBufferAllocator, NO_TYPE_HINT_MAPPING); + } + public List getColumns() { return columns; } 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 625825a1a..f134a144f 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 @@ -51,6 +51,8 @@ public abstract class AbstractBinaryFormatReader implements ClickHouseBinaryFormatReader { + public static final Map> NO_TYPE_HINT_MAPPING = null; + private static final Logger LOG = LoggerFactory.getLogger(AbstractBinaryFormatReader.class); protected InputStream input; @@ -63,8 +65,7 @@ public abstract class AbstractBinaryFormatReader implements ClickHouseBinaryForm private boolean hasNext = true; private boolean initialState = true; // reader is in initial state, no records have been read yet - protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema, - BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) { + protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema,BinaryStreamReader.ByteBufferAllocator byteBufferAllocator, Map> defaultTypeHintMap) { this.input = inputStream; Map settings = querySettings == null ? Collections.emptyMap() : querySettings.getAllSettings(); Boolean useServerTimeZone = (Boolean) settings.get(ClientConfigProperties.USE_SERVER_TIMEZONE.getKey()); @@ -75,7 +76,8 @@ protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings quer } boolean jsonAsString = MapUtils.getFlag(settings, ClientConfigProperties.serverSetting(ServerSettings.OUTPUT_FORMAT_BINARY_WRITE_JSON_AS_STRING), false); - this.binaryStreamReader = new BinaryStreamReader(inputStream, timeZone, LOG, byteBufferAllocator, jsonAsString); + this.binaryStreamReader = new BinaryStreamReader(inputStream, timeZone, LOG, byteBufferAllocator, jsonAsString, + defaultTypeHintMap); if (schema != null) { setSchema(schema); } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java index 577617678..4db8b2b16 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java @@ -43,6 +43,8 @@ */ public class BinaryStreamReader { + public static final Class NO_TYPE_HINT = null; + private final InputStream input; private final Logger log; @@ -53,6 +55,8 @@ public class BinaryStreamReader { private final boolean jsonAsString; + private final Class arrayDefaultTypeHint; + /** * Createa a BinaryStreamReader instance that will use the provided buffer allocator. * @@ -60,13 +64,18 @@ public class BinaryStreamReader { * @param timeZone - timezone to use for date and datetime values * @param log - logger * @param bufferAllocator - byte buffer allocator + * @param jsonAsString - use string to serialize/deserialize JSON columns + * @param typeHintMapping - what type use as hint if hint is not set or may not be known. */ - BinaryStreamReader(InputStream input, TimeZone timeZone, Logger log, ByteBufferAllocator bufferAllocator, boolean jsonAsString) { + BinaryStreamReader(InputStream input, TimeZone timeZone, Logger log, ByteBufferAllocator bufferAllocator, boolean jsonAsString, Map> typeHintMapping) { this.log = log == null ? NOPLogger.NOP_LOGGER : log; this.timeZone = timeZone; this.input = input; this.bufferAllocator = bufferAllocator; this.jsonAsString = jsonAsString; + + this.arrayDefaultTypeHint = typeHintMapping == null || + typeHintMapping.isEmpty()? NO_TYPE_HINT : typeHintMapping.get(ClickHouseDataType.Array); } /** @@ -217,6 +226,7 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce } // case Object: // deprecated https://clickhouse.com/docs/en/sql-reference/data-types/object-data-type case Array: + if (typeHint == null) { typeHint = arrayDefaultTypeHint;} return convertArray(readArray(actualColumn), typeHint); case Map: return (T) readMap(actualColumn); diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/PreparedStatementImpl.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/PreparedStatementImpl.java index 6566f7bb2..dc7007b3d 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/PreparedStatementImpl.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/PreparedStatementImpl.java @@ -784,6 +784,32 @@ private static String encodeObject(Object x) throws SQLException { } listString.append("]"); + return listString.toString(); + } else if (x.getClass().isArray()) { + StringBuilder listString = new StringBuilder(); + listString.append("["); + + + if (x.getClass().getComponentType().isPrimitive()) { + int len = java.lang.reflect.Array.getLength(x); + for (int i = 0; i < len; i++) { + if (i > 0) { + listString.append(", "); + } + listString.append(encodeObject(java.lang.reflect.Array.get(x, i))); + } + } else { + int i = 0; + for (Object item : (Object[]) x) { + if (i > 0) { + listString.append(", "); + } + listString.append(encodeObject(item)); + i++; + } + } + listString.append("]"); + return listString.toString(); } else if (x instanceof Collection) { StringBuilder listString = new StringBuilder(); diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/ResultSetImpl.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/ResultSetImpl.java index 9767a4640..5463beb7e 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/ResultSetImpl.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/ResultSetImpl.java @@ -35,6 +35,7 @@ import java.sql.Timestamp; import java.time.ZonedDateTime; import java.util.Calendar; +import java.util.Collection; import java.util.Map; public class ResultSetImpl implements ResultSet, JdbcV2Wrapper { diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java index fdc5f4f16..c9c7b9915 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcConfiguration.java @@ -2,6 +2,7 @@ import com.clickhouse.client.api.Client; import com.clickhouse.client.api.ClientConfigProperties; +import com.clickhouse.data.ClickHouseDataType; import com.clickhouse.jdbc.Driver; import com.google.common.collect.ImmutableMap; @@ -248,7 +249,8 @@ public String getDriverProperty(String key, String defaultValue) { public Client.Builder applyClientProperties(Client.Builder builder) { builder.addEndpoint(connectionUrl) - .setOptions(clientProperties); + .setOptions(clientProperties) + .typeHintMapping(defaultTypeHintMapping()); return builder; } @@ -268,4 +270,10 @@ public boolean isBetaFeatureEnabled(DriverProperties prop) { String value = driverProperties.getOrDefault(prop.getKey(), prop.getDefaultValue()); return Boolean.parseBoolean(value); } + + private Map> defaultTypeHintMapping() { + Map> mapping = new HashMap<>(); + mapping.put(ClickHouseDataType.Array, List.class); + return mapping; + } } 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 dd1865ff7..3a1ec6058 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java @@ -30,6 +30,7 @@ import java.time.ZonedDateTime; import java.util.GregorianCalendar; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Random; @@ -840,6 +841,62 @@ public void testMapTypes() throws SQLException { } } + + @Test(groups = { "integration" }) + public void testMapTypesWithArrayValues() throws SQLException { + runQuery("CREATE TABLE test_maps (order Int8, " + + "map Map(String, Array(Int32)), " + + "map2 Map(String, Array(Int32))" + + ") ENGINE = MergeTree ORDER BY ()"); + + // Insert random (valid) values + long seed = System.currentTimeMillis(); + Random rand = new Random(seed); + log.info("Random seed was: {}", seed); + + int mapSize = 3; + Map integerMap = new java.util.HashMap<>(mapSize); + Map integerMap2 = new java.util.HashMap<>(mapSize); + for (int i = 0; i < mapSize; i++) { + int[] array = new int[10]; + Integer[] array2 = new Integer[10]; + for (int j = 0; j < array.length; j++) { + array[j] = array2[j] = rand.nextInt(1000); + + } + integerMap.put("key" + i, array); + integerMap2.put("key" + i, array2); + } + + // Insert random (valid) values + try (Connection conn = getConnection()) { + try (PreparedStatement stmt = conn.prepareStatement("INSERT INTO test_maps VALUES ( 1, ?, ?)")) { + stmt.setObject(1, integerMap); + stmt.setObject(2, integerMap2); + stmt.executeUpdate(); + } + } + + // Check the results + try (Connection conn = getConnection()) { + try (Statement stmt = conn.createStatement()) { + try (ResultSet rs = stmt.executeQuery("SELECT * FROM test_maps ORDER BY order")) { + assertTrue(rs.next()); + Map mapResult = (Map) rs.getObject("map"); + assertEquals(mapResult.size(), mapSize); + for (String key: integerMap.keySet()) { + Object[] arrayResult = ((List) mapResult.get(key)).toArray(); + int[] array = integerMap.get(key); + assertEquals(arrayResult.length, array.length); + for (int i = 0; i < array.length; i++) { + assertEquals(String.valueOf(arrayResult[i]), String.valueOf(array[i])); + } + } + } + } + } + } + @Test(groups = { "integration" }) public void testNullableTypesSimpleStatement() throws SQLException { runQuery("CREATE TABLE test_nullable (order Int8, " From 0aaaf961478e503f776afb3dd0417ce912fd7465 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Fri, 27 Jun 2025 22:30:58 -0700 Subject: [PATCH 2/6] formatted test --- .../com/clickhouse/jdbc/DataTypeTests.java | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) 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 3a1ec6058..6d92a6c7c 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java @@ -679,7 +679,9 @@ public void testBooleanTypes() throws SQLException { @Test(groups = { "integration" }) public void testArrayTypes() throws SQLException { runQuery("CREATE TABLE test_arrays (order Int8, " - + "array Array(Int8), arraystr Array(String), arraytuple Array(Tuple(Int8, String)), arraydate Array(Date)" + + "array Array(Int8), arraystr Array(String), " + + "arraytuple Array(Tuple(Int8, String)), " + + "arraydate Array(Date)" + ") ENGINE = MergeTree ORDER BY ()"); // Insert random (valid) values @@ -707,7 +709,7 @@ public void testArrayTypes() throws SQLException { arraydate[i] = Date.valueOf(LocalDate.now().plusDays(rand.nextInt(100))); } - // Insert random (valid) values + // Insert using `Connection#createArrayOf` try (Connection conn = getConnection()) { try (PreparedStatement stmt = conn.prepareStatement("INSERT INTO test_arrays VALUES ( 1, ?, ?, ?, ?)")) { stmt.setArray(1, conn.createArrayOf("Int8", array)); @@ -718,6 +720,18 @@ public void testArrayTypes() throws SQLException { } } + // Insert using common java objects + final String INSERT_SQL = "INSERT INTO test_arrays VALUES ( 1, ?, ?, ?, ?)"; + try (Connection conn = getConnection()) { + try (PreparedStatement stmt = conn.prepareStatement(INSERT_SQL)) { + stmt.setObject(1, array); + stmt.setObject(2, arraystr); + stmt.setObject(3, arraytuple); + stmt.setObject(4, arraydate); + stmt.executeUpdate(); + } + } + // Check the results try (Connection conn = getConnection()) { try (Statement stmt = conn.createStatement()) { From 765524b81f9285ec977a1fd3905585847c04a834 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 1 Jul 2025 00:53:17 -0700 Subject: [PATCH 3/6] fixed tests --- .../com/clickhouse/client/api/Client.java | 20 +-- .../client/api/ClientConfigProperties.java | 34 ++++- .../clickhouse/jdbc/internal/JdbcUtils.java | 6 + .../com/clickhouse/jdbc/DataTypeTests.java | 116 +++++++++--------- 4 files changed, 96 insertions(+), 80 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java index 375e33665..451437d69 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java @@ -198,25 +198,7 @@ private Client(Set endpoints, Map configuration, this.serverVersion = configuration.getOrDefault(ClientConfigProperties.SERVER_VERSION.getKey(), "unknown"); - this.typeHintMapping = translateTypeHintMapping(configuration.get(ClientConfigProperties.TYPE_HINT_MAPPING.getKey())); - } - - private Map> translateTypeHintMapping(String mappingStr) { - if (mappingStr == null || mappingStr.isEmpty()) { - return AbstractBinaryFormatReader.NO_TYPE_HINT_MAPPING; - } - - Map mapping= ClientConfigProperties.toKeyValuePairs(mappingStr); - Map> hintMapping = new HashMap<>(); - try { - for (Map.Entry entry : mapping.entrySet()) { - hintMapping.put(ClickHouseDataType.of(entry.getKey()), - Class.forName(entry.getValue())); - } - } catch (ClassNotFoundException e) { - throw new ClientMisconfigurationException("Failed to translate type-hint mapping", e); - } - return hintMapping; + this.typeHintMapping = (Map>) this.configuration.get(ClientConfigProperties.TYPE_HINT_MAPPING.getKey()); } /** diff --git a/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java b/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java index a0f408edb..a2eb1cdf1 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java @@ -1,6 +1,8 @@ package com.clickhouse.client.api; +import com.clickhouse.client.api.data_formats.internal.AbstractBinaryFormatReader; import com.clickhouse.client.api.internal.ClickHouseLZ4OutputStream; +import com.clickhouse.data.ClickHouseDataType; import com.clickhouse.data.ClickHouseFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -286,6 +288,10 @@ public Object parseValue(String value) { return TimeZone.getTimeZone(value); } + if (valueType.equals(Map.class)) { + return toKeyValuePairs(value); + } + return null; } @@ -309,7 +315,15 @@ public static Map parseConfigMap(Map configMap) for (ClientConfigProperties config : ClientConfigProperties.values()) { String value = tmpMap.remove(config.getKey()); if (value != null) { - parsedConfig.put(config.getKey(), config.parseValue(value)); + Object parsedValue; + switch (config) { + case TYPE_HINT_MAPPING: + parsedValue = translateTypeHintMapping(value); + break; + default: + parsedValue = config.parseValue(value); + } + parsedConfig.put(config.getKey(), parsedValue); } } @@ -388,4 +402,22 @@ public static String mapToString(Map map, Function valueCon } return sb.toString(); } + + public static Map> translateTypeHintMapping(String mappingStr) { + if (mappingStr == null || mappingStr.isEmpty()) { + return AbstractBinaryFormatReader.NO_TYPE_HINT_MAPPING; + } + + Map mapping= ClientConfigProperties.toKeyValuePairs(mappingStr); + Map> hintMapping = new HashMap<>(); + try { + for (Map.Entry entry : mapping.entrySet()) { + hintMapping.put(ClickHouseDataType.of(entry.getKey()), + Class.forName(entry.getValue())); + } + } catch (ClassNotFoundException e) { + throw new ClientMisconfigurationException("Failed to translate type-hint mapping", e); + } + return hintMapping; + } } diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcUtils.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcUtils.java index abf7f2b69..26d655d67 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcUtils.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcUtils.java @@ -229,6 +229,12 @@ public static Object convert(Object value, Class type, ClickHouseColumn colum return new Array(convertList(((BinaryStreamReader.ArrayValue) value).asList(), JdbcUtils.convertToJavaClass(column.getArrayBaseColumn().getDataType())), "Object", JDBCType.JAVA_OBJECT.getVendorTypeNumber()); } return new Array(((BinaryStreamReader.ArrayValue) value).asList(), "Object", JDBCType.JAVA_OBJECT.getVendorTypeNumber()); + } else if (type == java.sql.Array.class && value instanceof List) { + System.out.println(value); + if (column != null && column.getArrayBaseColumn() != null) { + return new Array(convertList(((List) value), JdbcUtils.convertToJavaClass(column.getArrayBaseColumn().getDataType())), "Object", JDBCType.JAVA_OBJECT.getVendorTypeNumber()); + } + return new Array((List) value, "Object", JDBCType.JAVA_OBJECT.getVendorTypeNumber()); } else if (type == Inet4Address.class && value instanceof Inet6Address) { // Convert Inet6Address to Inet4Address return Inet4Address.getByName(value.toString()); 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 3e0ca83e5..bf8f51362 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java @@ -762,22 +762,22 @@ public void testArrayTypes() throws SQLException { Random rand = new Random(seed); log.info("Random seed was: {}", seed); - Integer[] array = new Integer[rand.nextInt(10)]; + Integer[] array = new Integer[10]; for (int i = 0; i < array.length; i++) { array[i] = rand.nextInt(256) - 128; } - String[] arraystr = new String[rand.nextInt(10)]; + String[] arraystr = new String[10]; for (int i = 0; i < arraystr.length; i++) { arraystr[i] = "string" + rand.nextInt(1000); } - Tuple[] arraytuple = new Tuple[rand.nextInt(10) + 1]; + Tuple[] arraytuple = new Tuple[10]; for (int i = 0; i < arraytuple.length; i++) { arraytuple[i] = new Tuple(rand.nextInt(256) - 128, "string" + rand.nextInt(1000)); } - Date[] arraydate = new Date[rand.nextInt(10)]; + Date[] arraydate = new Date[10]; for (int i = 0; i < arraydate.length; i++) { arraydate[i] = Date.valueOf(LocalDate.now().plusDays(rand.nextInt(100))); } @@ -795,7 +795,7 @@ public void testArrayTypes() throws SQLException { // Insert using common java objects final String INSERT_SQL = "INSERT INTO test_arrays VALUES ( 1, ?, ?, ?, ?)"; - try (Connection conn = getConnection()) { + try (Connection conn = getJdbcConnection()) { try (PreparedStatement stmt = conn.prepareStatement(INSERT_SQL)) { stmt.setObject(1, array); stmt.setObject(2, arraystr); @@ -810,65 +810,60 @@ public void testArrayTypes() throws SQLException { try (Statement stmt = conn.createStatement()) { try (ResultSet rs = stmt.executeQuery("SELECT * FROM test_arrays ORDER BY order")) { assertTrue(rs.next()); - Object[] arrayResult = (Object[]) rs.getArray("array").getArray(); - assertEquals(arrayResult.length, array.length); - for (int i = 0; i < array.length; i++) { - assertEquals(String.valueOf(arrayResult[i]), String.valueOf(array[i])); - } + { + Object[] arrayResult = (Object[]) rs.getArray("array").getArray(); + assertEquals(arrayResult.length, array.length); + for (int i = 0; i < array.length; i++) { + assertEquals(String.valueOf(arrayResult[i]), String.valueOf(array[i])); + } - Object[] arraystrResult = (Object[]) rs.getArray("arraystr").getArray(); - assertEquals(arraystrResult.length, arraystr.length); - for (int i = 0; i < arraystr.length; i++) { - assertEquals(arraystrResult[i], arraystr[i]); - } - Object[] arraytupleResult = (Object[]) rs.getArray("arraytuple").getArray(); - assertEquals(arraytupleResult.length, arraytuple.length); - for (int i = 0; i < arraytuple.length; i++) { - Tuple tuple = arraytuple[i]; - Tuple tupleResult = new Tuple(((Object[]) arraytupleResult[i])); - assertEquals(String.valueOf(tupleResult.getValue(0)), String.valueOf(tuple.getValue(0))); - assertEquals(String.valueOf(tupleResult.getValue(1)), String.valueOf(tuple.getValue(1))); - } + Object[] arraystrResult = (Object[]) rs.getArray("arraystr").getArray(); + assertEquals(arraystrResult.length, arraystr.length); + for (int i = 0; i < arraystr.length; i++) { + assertEquals(arraystrResult[i], arraystr[i]); + } + Object[] arraytupleResult = (Object[]) rs.getArray("arraytuple").getArray(); + assertEquals(arraytupleResult.length, arraytuple.length); + for (int i = 0; i < arraytuple.length; i++) { + Tuple tuple = arraytuple[i]; + Tuple tupleResult = new Tuple(((Object[]) arraytupleResult[i])); + assertEquals(String.valueOf(tupleResult.getValue(0)), String.valueOf(tuple.getValue(0))); + assertEquals(String.valueOf(tupleResult.getValue(1)), String.valueOf(tuple.getValue(1))); + } - Object[] arraydateResult = (Object[]) rs.getArray("arraydate").getArray(); - assertEquals(arraydateResult.length, arraydate.length); - for (int i = 0; i < arraydate.length; i++) { - assertEquals(String.valueOf(arraydateResult[i]), String.valueOf(arraydate[i])); + Object[] arraydateResult = (Object[]) rs.getArray("arraydate").getArray(); + assertEquals(arraydateResult.length, arraydate.length); + for (int i = 0; i < arraydate.length; i++) { + assertEquals(String.valueOf(arraydateResult[i]), String.valueOf(arraydate[i])); + } } - assertFalse(rs.next()); - } - } - } - - // Check the results with getObject - try (Connection conn = getJdbcConnection()) { - try (Statement stmt = conn.createStatement()) { - try (ResultSet rs = stmt.executeQuery("SELECT * FROM test_arrays ORDER BY order")) { assertTrue(rs.next()); - Object[] arrayResult = (Object[]) ((Array) rs.getObject("array")).getArray(); - assertEquals(arrayResult.length, array.length); - for (int i = 0; i < array.length; i++) { - assertEquals(String.valueOf(arrayResult[i]), String.valueOf(array[i])); - } + { + Object[] arrayResult = (Object[]) ((Array) rs.getObject("array")).getArray(); + assertEquals(arrayResult.length, array.length); + for (int i = 0; i < array.length; i++) { + assertEquals(String.valueOf(arrayResult[i]), String.valueOf(array[i])); + } - Object[] arraystrResult = (Object[]) ((Array) rs.getObject("arraystr")).getArray(); - assertEquals(arraystrResult.length, arraystr.length); - for (int i = 0; i < arraystr.length; i++) { - assertEquals(arraystrResult[i], arraystr[i]); - } - Object[] arraytupleResult = (Object[]) ((Array) rs.getObject("arraytuple")).getArray(); - assertEquals(arraytupleResult.length, arraytuple.length); - for (int i = 0; i < arraytuple.length; i++) { - Tuple tuple = arraytuple[i]; - Tuple tupleResult = new Tuple(((Object[]) arraytupleResult[i])); - assertEquals(String.valueOf(tupleResult.getValue(0)), String.valueOf(tuple.getValue(0))); - assertEquals(String.valueOf(tupleResult.getValue(1)), String.valueOf(tuple.getValue(1))); - } + Object[] arraystrResult = (Object[]) ((Array) rs.getObject("arraystr")).getArray(); + assertEquals(arraystrResult.length, arraystr.length); + for (int i = 0; i < arraystr.length; i++) { + assertEquals(arraystrResult[i], arraystr[i]); + } + Object[] arraytupleResult = (Object[]) ((Array) rs.getObject("arraytuple")).getArray(); + assertEquals(arraytupleResult.length, arraytuple.length); + for (int i = 0; i < arraytuple.length; i++) { + Tuple tuple = arraytuple[i]; + Tuple tupleResult = new Tuple(((Object[]) arraytupleResult[i])); + assertEquals(String.valueOf(tupleResult.getValue(0)), String.valueOf(tuple.getValue(0))); + assertEquals(String.valueOf(tupleResult.getValue(1)), String.valueOf(tuple.getValue(1))); + } - Object[] arraydateResult = (Object[]) ((Array) rs.getObject("arraydate")).getArray(); - assertEquals(arraydateResult.length, arraydate.length); - for (int i = 0; i < arraydate.length; i++) { - assertEquals(arraydateResult[i], arraydate[i]); + Object[] arraydateResult = (Object[]) ((Array) rs.getObject("arraydate")).getArray(); + assertEquals(arraydateResult.length, arraydate.length); + for (int i = 0; i < arraydate.length; i++) { + assertEquals(arraydateResult[i], arraydate[i]); + } } assertFalse(rs.next()); } @@ -931,6 +926,7 @@ public void testMapTypes() throws SQLException { @Test(groups = { "integration" }) public void testMapTypesWithArrayValues() throws SQLException { + runQuery("DROP TABLE test_maps;"); runQuery("CREATE TABLE test_maps (order Int8, " + "map Map(String, Array(Int32)), " + "map2 Map(String, Array(Int32))" @@ -956,7 +952,7 @@ public void testMapTypesWithArrayValues() throws SQLException { } // Insert random (valid) values - try (Connection conn = getConnection()) { + try (Connection conn = getJdbcConnection()) { try (PreparedStatement stmt = conn.prepareStatement("INSERT INTO test_maps VALUES ( 1, ?, ?)")) { stmt.setObject(1, integerMap); stmt.setObject(2, integerMap2); @@ -965,7 +961,7 @@ public void testMapTypesWithArrayValues() throws SQLException { } // Check the results - try (Connection conn = getConnection()) { + try (Connection conn = getJdbcConnection()) { try (Statement stmt = conn.createStatement()) { try (ResultSet rs = stmt.executeQuery("SELECT * FROM test_maps ORDER BY order")) { assertTrue(rs.next()); From bc4c13b0964af571090ffbb592727f08264363c5 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 8 Jul 2025 10:33:10 -0700 Subject: [PATCH 4/6] added a test for toKeyValuePairs() --- .../client/api/ClientConfigProperties.java | 3 ++ .../internal/AbstractBinaryFormatReader.java | 2 +- .../api/ClientConfigPropertiesTest.java | 34 +++++++++++++++++++ .../clickhouse/jdbc/internal/JdbcUtils.java | 1 - 4 files changed, 38 insertions(+), 2 deletions(-) create mode 100644 client-v2/src/test/java/com/clickhouse/client/api/ClientConfigPropertiesTest.java diff --git a/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java b/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java index 9836c6f98..cf8caee8f 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java @@ -343,6 +343,8 @@ public static Map parseConfigMap(Map configMap) /** * Converts given string to key value pairs. + * This is very simple implementation that do not handle edge cases like + * {@code k1=v1, ,k2=v2} * * @param str string * @return non-null key value pairs @@ -393,6 +395,7 @@ public static Map toKeyValuePairs(String str) { } + public static String mapToString(Map map, Function valueConverter) { StringBuilder sb = new StringBuilder(); for (Map.Entry entry : map.entrySet()) { 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 9454b47ce..1da726707 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 @@ -51,7 +51,7 @@ public abstract class AbstractBinaryFormatReader implements ClickHouseBinaryFormatReader { - public static final Map> NO_TYPE_HINT_MAPPING = null; + public static final Map> NO_TYPE_HINT_MAPPING = Collections.emptyMap(); private static final Logger LOG = LoggerFactory.getLogger(AbstractBinaryFormatReader.class); diff --git a/client-v2/src/test/java/com/clickhouse/client/api/ClientConfigPropertiesTest.java b/client-v2/src/test/java/com/clickhouse/client/api/ClientConfigPropertiesTest.java new file mode 100644 index 000000000..f805ee65b --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/api/ClientConfigPropertiesTest.java @@ -0,0 +1,34 @@ +package com.clickhouse.client.api; + + +import org.testng.Assert; +import org.testng.annotations.Test; + +import java.util.Map; + + +public class ClientConfigPropertiesTest { + + @Test(groups = {"unit"}) + public void testToKeyValuePairs() { + + Map map = ClientConfigProperties.toKeyValuePairs("key1=value1,key2=value2"); + Assert.assertEquals(map.size(), 2); + Assert.assertEquals(map.get("key1"), "value1"); + Assert.assertEquals(map.get("key2"), "value2"); + + map = ClientConfigProperties.toKeyValuePairs("key1=value1, key2 = value2"); + Assert.assertEquals(map.size(), 2); + Assert.assertEquals(map.get("key1"), "value1"); + Assert.assertEquals(map.get("key2"), "value2"); + + map = ClientConfigProperties.toKeyValuePairs("key1"); + Assert.assertEquals(map.size(), 0); + + // TODO: improve implementation +// map = ClientConfigProperties.toKeyValuePairs("key1=value1, ,key2=value2"); +// Assert.assertEquals(map.size(), 2); +// Assert.assertEquals(map.get("key1"), "value1"); +// Assert.assertEquals(map.get("key2"), "value2"); + } +} \ No newline at end of file diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcUtils.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcUtils.java index 26d655d67..8da2bc8a6 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcUtils.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/internal/JdbcUtils.java @@ -230,7 +230,6 @@ public static Object convert(Object value, Class type, ClickHouseColumn colum } return new Array(((BinaryStreamReader.ArrayValue) value).asList(), "Object", JDBCType.JAVA_OBJECT.getVendorTypeNumber()); } else if (type == java.sql.Array.class && value instanceof List) { - System.out.println(value); if (column != null && column.getArrayBaseColumn() != null) { return new Array(convertList(((List) value), JdbcUtils.convertToJavaClass(column.getArrayBaseColumn().getDataType())), "Object", JDBCType.JAVA_OBJECT.getVendorTypeNumber()); } From 38991f1167227ae115814de15a571ef38c6ef70f Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 8 Jul 2025 11:56:57 -0700 Subject: [PATCH 5/6] fixed primitive arrays. added more tests --- .../internal/AbstractBinaryFormatReader.java | 86 +++++++++++++------ .../internal/MapBackedRecord.java | 2 + .../client/api/internal/StopWatch.java | 4 + .../clickhouse/client/query/QueryTests.java | 32 ++++++- 4 files changed, 99 insertions(+), 25 deletions(-) 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 1da726707..ab11b731b 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 @@ -8,8 +8,8 @@ import com.clickhouse.client.api.metadata.NoSuchColumnException; import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.NullValueException; -import com.clickhouse.client.api.serde.POJOFieldDeserializer; import com.clickhouse.client.api.query.QuerySettings; +import com.clickhouse.client.api.serde.POJOFieldDeserializer; import com.clickhouse.data.ClickHouseColumn; import com.clickhouse.data.ClickHouseDataType; import com.clickhouse.data.value.ClickHouseBitmap; @@ -24,6 +24,7 @@ import java.io.IOException; import java.io.InputStream; import java.lang.ref.WeakReference; +import java.lang.reflect.Array; import java.math.BigDecimal; import java.math.BigInteger; import java.net.Inet4Address; @@ -524,23 +525,36 @@ public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) { @Override public List getList(String colName) { - try { - BinaryStreamReader.ArrayValue array = readValue(colName); - return array.asList(); - } catch (ClassCastException e) { - throw new ClientException("Column is not of array type", e); + Object value = readValue(colName); + if (value instanceof BinaryStreamReader.ArrayValue) { + return ((BinaryStreamReader.ArrayValue) value).asList(); + } else if (value instanceof List) { + return (List) value; + } else { + throw new ClientException("Column is not of array type"); } } - private T getPrimitiveArray(String colName) { + private T getPrimitiveArray(String colName, Class componentType) { try { - BinaryStreamReader.ArrayValue array = readValue(colName); - if (array.itemType.isPrimitive()) { - return (T) array.array; - } else { - throw new ClientException("Array is not of primitive type"); + Object value = readValue(colName); + if (value instanceof BinaryStreamReader.ArrayValue) { + BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) value; + if (array.itemType.isPrimitive()) { + return (T) array.array; + } else { + throw new ClientException("Array is not of primitive type"); + } + } else if (value instanceof List) { + List list = (List) value; + Object array = Array.newInstance(componentType, list.size()); + for (int i = 0; i < list.size(); i++) { + Array.set(array, i, list.get(i)); + } + return (T)array; } + throw new ClientException("Column is not of array type"); } catch (ClassCastException e) { throw new ClientException("Column is not of array type", e); } @@ -548,32 +562,56 @@ private T getPrimitiveArray(String colName) { @Override public byte[] getByteArray(String colName) { - return getPrimitiveArray(colName); + try { + return getPrimitiveArray(colName, byte.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } } @Override public int[] getIntArray(String colName) { - return getPrimitiveArray(colName); + try { + return getPrimitiveArray(colName, int.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } } @Override public long[] getLongArray(String colName) { - return getPrimitiveArray(colName); + try { + return getPrimitiveArray(colName, long.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } } @Override public float[] getFloatArray(String colName) { - return getPrimitiveArray(colName); + try { + return getPrimitiveArray(colName, float.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } } @Override public double[] getDoubleArray(String colName) { - return getPrimitiveArray(colName); + try { + return getPrimitiveArray(colName, double.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } } @Override public boolean[] getBooleanArray(String colName) { - return getPrimitiveArray(colName); + try { + return getPrimitiveArray(colName, boolean.class); + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value cannot be converted to an array of primitives", e); + } } @Override @@ -693,32 +731,32 @@ public List getList(int index) { @Override public byte[] getByteArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); + return getByteArray(schema.columnIndexToName(index)); } @Override public int[] getIntArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); + return getIntArray(schema.columnIndexToName(index)); } @Override public long[] getLongArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); + return getLongArray(schema.columnIndexToName(index)); } @Override public float[] getFloatArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); + return getFloatArray(schema.columnIndexToName(index)); } @Override public double[] getDoubleArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); + return getDoubleArray(schema.columnIndexToName(index)); } @Override public boolean[] getBooleanArray(int index) { - return getPrimitiveArray(schema.columnIndexToName(index)); + return getBooleanArray(schema.columnIndexToName(index)); } @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 f3e5d0856..ef24ae782 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 @@ -199,6 +199,8 @@ public List getList(String colName) { Object value = readValue(colName); if (value instanceof BinaryStreamReader.ArrayValue) { return ((BinaryStreamReader.ArrayValue) value).asList(); + } else if (value instanceof List) { + return (List) value; } else { throw new ClientException("Column is not of array type"); } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/StopWatch.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/StopWatch.java index ac686f6ef..077286141 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/StopWatch.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/StopWatch.java @@ -33,6 +33,10 @@ public long getElapsedTime() { return TimeUnit.NANOSECONDS.toMillis(elapsedNanoTime); } + public long getElapsedNanos() { + return elapsedNanoTime; + } + @Override public String toString() { return "{" + 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 749985d12..384639c25 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 @@ -18,6 +18,7 @@ import com.clickhouse.client.api.insert.InsertResponse; import com.clickhouse.client.api.insert.InsertSettings; import com.clickhouse.client.api.internal.ServerSettings; +import com.clickhouse.client.api.internal.StopWatch; import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.metrics.ClientMetrics; import com.clickhouse.client.api.metrics.OperationMetrics; @@ -571,6 +572,35 @@ public void testArrayValues() throws Exception { Assert.assertEquals(reader.getList("col5"), ((List)data.get(0).get("col5"))); } + + @Test + public void testPrimitiveArrays() throws Exception { + final String table = "primitive_arrays_test_table"; + + try (QueryResponse response = client.query("SELECT [1, 2, 3]::Array(UInt32) as arr").get()) { + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response); + + reader.next(); + Assert.assertEquals(reader.readValue(1).getClass(), BinaryStreamReader.ArrayValue.class); + Assert.assertEquals(reader.getList(1).get(0).getClass(), Long.class); + Assert.assertEquals(reader.getList(1), Arrays.asList((long)1, (long)2, (long)3)); + Assert.assertEquals(reader.getLongArray(1), new long[]{1, 2, 3}); + Assert.assertThrows(ClientException.class, () -> reader.getIntArray(1)); + } + + try (Client client1 = newClient().typeHintMapping(Collections.singletonMap(ClickHouseDataType.Array, List.class)).build(); + QueryResponse response = client1.query("SELECT [1, 2, 3]::Array(UInt32) as arr").get()) { + ClickHouseBinaryFormatReader reader = client1.newBinaryFormatReader(response); + + reader.next(); + Assert.assertEquals(reader.readValue(1).getClass(), ArrayList.class); + Assert.assertEquals(reader.getList(1).get(0).getClass(), Long.class); + Assert.assertEquals(reader.getList(1), Arrays.asList((long)1, (long)2, (long)3)); + Assert.assertEquals(reader.getLongArray(1), new long[]{1, 2, 3}); + Assert.assertThrows(ClientException.class, () -> reader.getIntArray(1)); + } + } + @Test public void testArraysAsList() { GenericRecord record = @@ -1405,7 +1435,7 @@ public void testQueryMetrics() throws Exception { Assert.assertEquals(metrics.getMetric(ServerMetrics.NUM_ROWS_READ).getLong(), rowsToInsert); // 10 rows in the table Assert.assertEquals(metrics.getMetric(ServerMetrics.RESULT_ROWS).getLong(), rowsToInsert); Assert.assertEquals(response.getReadRows(), rowsToInsert); - Assert.assertTrue(metrics.getMetric(ClientMetrics.OP_DURATION).getLong() > 0); + Assert.assertTrue(((StopWatch)metrics.getMetric(ClientMetrics.OP_DURATION)).getElapsedNanos() > 0); Assert.assertEquals(metrics.getQueryId(), uuid); Assert.assertEquals(response.getQueryId(), uuid); } From 16611779290481ff7337f9705447421e362433c5 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 8 Jul 2025 12:07:46 -0700 Subject: [PATCH 6/6] fixed array tests --- jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 fd02d792d..16e0b5890 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java @@ -825,7 +825,7 @@ public void testArrayTypes() throws SQLException { } // Insert using common java objects - final String INSERT_SQL = "INSERT INTO test_arrays VALUES ( 1, ?, ?, ?, ?)"; + final String INSERT_SQL = "INSERT INTO test_arrays VALUES ( 2, ?, ?, ?, ?)"; try (Connection conn = getJdbcConnection()) { try (PreparedStatement stmt = conn.prepareStatement(INSERT_SQL)) { stmt.setObject(1, array);