Skip to content

Commit 04e2c1d

Browse files
authored
Merge pull request #2476 from ClickHouse/jdbc_fix_nested_types
[client-v2, jdbc-v2] Configurable type hinting
2 parents b7ce67c + 1661177 commit 04e2c1d

File tree

17 files changed

+502
-111
lines changed

17 files changed

+502
-111
lines changed

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

Lines changed: 25 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,7 @@
77
import com.clickhouse.client.api.data_formats.RowBinaryFormatReader;
88
import com.clickhouse.client.api.data_formats.RowBinaryWithNamesAndTypesFormatReader;
99
import com.clickhouse.client.api.data_formats.RowBinaryWithNamesFormatReader;
10+
import com.clickhouse.client.api.data_formats.internal.AbstractBinaryFormatReader;
1011
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
1112
import com.clickhouse.client.api.data_formats.internal.MapBackedRecord;
1213
import com.clickhouse.client.api.data_formats.internal.ProcessParser;
@@ -36,7 +37,9 @@
3637
import com.clickhouse.client.api.transport.Endpoint;
3738
import com.clickhouse.client.api.transport.HttpEndpoint;
3839
import com.clickhouse.client.config.ClickHouseClientOption;
40+
import com.clickhouse.config.ClickHouseOption;
3941
import com.clickhouse.data.ClickHouseColumn;
42+
import com.clickhouse.data.ClickHouseDataType;
4043
import com.clickhouse.data.ClickHouseFormat;
4144
import com.google.common.collect.ImmutableList;
4245
import net.jpountz.lz4.LZ4Factory;
@@ -131,6 +134,8 @@ public class Client implements AutoCloseable {
131134

132135
private final Map<String, Boolean> tableSchemaHasDefaults = new ConcurrentHashMap<>();
133136

137+
private final Map<ClickHouseDataType, Class<?>> typeHintMapping;
138+
134139
// Server context
135140
private String serverVersion;
136141
private Object metricsRegistry;
@@ -192,6 +197,8 @@ private Client(Set<String> endpoints, Map<String,String> configuration,
192197
}
193198

194199
this.serverVersion = configuration.getOrDefault(ClientConfigProperties.SERVER_VERSION.getKey(), "unknown");
200+
201+
this.typeHintMapping = (Map<ClickHouseDataType, Class<?>>) this.configuration.get(ClientConfigProperties.TYPE_HINT_MAPPING.getKey());
195202
}
196203

197204
/**
@@ -1004,6 +1011,20 @@ public Builder setServerVersion(String serverVersion) {
10041011
return this;
10051012
}
10061013

1014+
/**
1015+
* Defines mapping between ClickHouse data type and target Java type
1016+
* Used by binary readers to convert values into desired Java type.
1017+
* @param typeHintMapping - map between ClickHouse data type and Java class
1018+
* @return this builder instance
1019+
*/
1020+
public Builder typeHintMapping(Map<ClickHouseDataType, Class<?>> typeHintMapping) {
1021+
this.configuration.put(ClientConfigProperties.TYPE_HINT_MAPPING.getKey(),
1022+
ClientConfigProperties.mapToString(typeHintMapping, (v) -> {
1023+
return ((Class<?>) v).getName();
1024+
}));
1025+
return this;
1026+
}
1027+
10071028
public Client build() {
10081029
// check if endpoint are empty. so can not initiate client
10091030
if (this.endpoints.isEmpty()) {
@@ -1919,23 +1940,20 @@ public ClickHouseBinaryFormatReader newBinaryFormatReader(QueryResponse response
19191940
BinaryStreamReader.ByteBufferAllocator byteBufferPool = useCachingBufferAllocator ?
19201941
new BinaryStreamReader.CachingByteBufferAllocator() :
19211942
new BinaryStreamReader.DefaultByteBufferAllocator();
1922-
19231943
switch (response.getFormat()) {
19241944
case Native:
19251945
reader = new NativeFormatReader(response.getInputStream(), response.getSettings(),
1926-
byteBufferPool);
1946+
byteBufferPool, typeHintMapping);
19271947
break;
19281948
case RowBinaryWithNamesAndTypes:
1929-
reader = new RowBinaryWithNamesAndTypesFormatReader(response.getInputStream(), response.getSettings(),
1930-
byteBufferPool);
1949+
reader = new RowBinaryWithNamesAndTypesFormatReader(response.getInputStream(), response.getSettings(), byteBufferPool, typeHintMapping);
19311950
break;
19321951
case RowBinaryWithNames:
1933-
reader = new RowBinaryWithNamesFormatReader(response.getInputStream(), response.getSettings(), schema,
1934-
byteBufferPool);
1952+
reader = new RowBinaryWithNamesFormatReader(response.getInputStream(), response.getSettings(), schema, byteBufferPool, typeHintMapping);
19351953
break;
19361954
case RowBinary:
19371955
reader = new RowBinaryFormatReader(response.getInputStream(), response.getSettings(), schema,
1938-
byteBufferPool);
1956+
byteBufferPool, typeHintMapping);
19391957
break;
19401958
default:
19411959
throw new IllegalArgumentException("Binary readers doesn't support format: " + response.getFormat());

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

Lines changed: 109 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,8 @@
11
package com.clickhouse.client.api;
22

3+
import com.clickhouse.client.api.data_formats.internal.AbstractBinaryFormatReader;
34
import com.clickhouse.client.api.internal.ClickHouseLZ4OutputStream;
5+
import com.clickhouse.data.ClickHouseDataType;
46
import com.clickhouse.data.ClickHouseFormat;
57
import org.slf4j.Logger;
68
import org.slf4j.LoggerFactory;
@@ -9,11 +11,14 @@
911
import java.util.Arrays;
1012
import java.util.Collection;
1113
import java.util.Collections;
14+
import java.util.LinkedHashMap;
1215
import java.util.HashMap;
1316
import java.util.HashSet;
1417
import java.util.List;
1518
import java.util.Locale;
1619
import java.util.Map;
20+
import java.util.function.Function;
21+
import java.util.Map;
1722
import java.util.TimeZone;
1823
import java.util.function.Consumer;
1924
import java.util.stream.Collectors;
@@ -168,6 +173,11 @@ public Object parseValue(String value) {
168173

169174
BINARY_READER_USE_PREALLOCATED_BUFFERS("client_allow_binary_reader_to_reuse_buffers", Boolean.class, "false"),
170175

176+
/**
177+
* Defines mapping between ClickHouse data type and target Java type
178+
* Used by binary readers to convert values into desired Java type.
179+
*/
180+
TYPE_HINT_MAPPING("type_hint_mapping", Map.class),
171181
;
172182

173183
private static final Logger LOG = LoggerFactory.getLogger(ClientConfigProperties.class);
@@ -278,6 +288,10 @@ public Object parseValue(String value) {
278288
return TimeZone.getTimeZone(value);
279289
}
280290

291+
if (valueType.equals(Map.class)) {
292+
return toKeyValuePairs(value);
293+
}
294+
281295
return null;
282296
}
283297

@@ -301,7 +315,15 @@ public static Map<String, Object> parseConfigMap(Map<String, String> configMap)
301315
for (ClientConfigProperties config : ClientConfigProperties.values()) {
302316
String value = tmpMap.remove(config.getKey());
303317
if (value != null) {
304-
parsedConfig.put(config.getKey(), config.parseValue(value));
318+
Object parsedValue;
319+
switch (config) {
320+
case TYPE_HINT_MAPPING:
321+
parsedValue = translateTypeHintMapping(value);
322+
break;
323+
default:
324+
parsedValue = config.parseValue(value);
325+
}
326+
parsedConfig.put(config.getKey(), parsedValue);
305327
}
306328
}
307329

@@ -317,4 +339,90 @@ public static Map<String, Object> parseConfigMap(Map<String, String> configMap)
317339

318340
return parsedConfig;
319341
}
342+
343+
344+
/**
345+
* Converts given string to key value pairs.
346+
* This is very simple implementation that do not handle edge cases like
347+
* {@code k1=v1, ,k2=v2}
348+
*
349+
* @param str string
350+
* @return non-null key value pairs
351+
*/
352+
public static Map<String, String> toKeyValuePairs(String str) {
353+
if (str == null || str.isEmpty()) {
354+
return Collections.emptyMap();
355+
}
356+
357+
Map<String, String> map = new LinkedHashMap<>();
358+
String key = null;
359+
StringBuilder builder = new StringBuilder();
360+
for (int i = 0, len = str.length(); i < len; i++) {
361+
char ch = str.charAt(i);
362+
if (ch == '\\' && i + 1 < len) {
363+
ch = str.charAt(++i);
364+
builder.append(ch);
365+
continue;
366+
}
367+
368+
if (Character.isWhitespace(ch)) {
369+
if (builder.length() > 0) {
370+
builder.append(ch);
371+
}
372+
} else if (ch == '=' && key == null) {
373+
key = builder.toString().trim();
374+
builder.setLength(0);
375+
} else if (ch == ',' && key != null) {
376+
String value = builder.toString().trim();
377+
builder.setLength(0);
378+
if (!key.isEmpty() && !value.isEmpty()) {
379+
map.put(key, value);
380+
}
381+
key = null;
382+
} else {
383+
builder.append(ch);
384+
}
385+
}
386+
387+
if (key != null && builder.length() > 0) {
388+
String value = builder.toString().trim();
389+
if (!key.isEmpty() && !value.isEmpty()) {
390+
map.put(key, value);
391+
}
392+
}
393+
394+
return Collections.unmodifiableMap(map);
395+
}
396+
397+
398+
399+
public static String mapToString(Map<?,?> map, Function<Object, String> valueConverter) {
400+
StringBuilder sb = new StringBuilder();
401+
for (Map.Entry<?, ?> entry : map.entrySet()) {
402+
sb.append(entry.getKey()).append("=").append(valueConverter.apply(entry.getValue())).append(",");
403+
}
404+
405+
if (sb.length() > 0) {
406+
sb.setLength(sb.length() - 1);
407+
}
408+
return sb.toString();
409+
}
410+
411+
public static Map<ClickHouseDataType, Class<?>> translateTypeHintMapping(String mappingStr) {
412+
if (mappingStr == null || mappingStr.isEmpty()) {
413+
return AbstractBinaryFormatReader.NO_TYPE_HINT_MAPPING;
414+
}
415+
416+
Map<String, String> mapping= ClientConfigProperties.toKeyValuePairs(mappingStr);
417+
Map<ClickHouseDataType, Class<?>> hintMapping = new HashMap<>();
418+
try {
419+
for (Map.Entry<String, String> entry : mapping.entrySet()) {
420+
hintMapping.put(ClickHouseDataType.of(entry.getKey()),
421+
Class.forName(entry.getValue()));
422+
}
423+
} catch (ClassNotFoundException e) {
424+
throw new ClientMisconfigurationException("Failed to translate type-hint mapping", e);
425+
}
426+
return hintMapping;
427+
}
320428
}

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

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,7 @@
66
import com.clickhouse.client.api.metadata.TableSchema;
77
import com.clickhouse.client.api.query.QuerySettings;
88
import com.clickhouse.data.ClickHouseColumn;
9+
import com.clickhouse.data.ClickHouseDataType;
910

1011
import java.io.EOFException;
1112
import java.io.IOException;
@@ -26,15 +27,21 @@ public class NativeFormatReader extends AbstractBinaryFormatReader {
2627
private int blockRowIndex;
2728

2829
public NativeFormatReader(InputStream inputStream, QuerySettings settings,
29-
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) {
30-
super(inputStream, settings, null, byteBufferAllocator);
30+
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator,
31+
Map<ClickHouseDataType, Class<?>> typeHintMapping) {
32+
super(inputStream, settings, null, byteBufferAllocator, typeHintMapping);
3133
try {
3234
readBlock();
3335
} catch (IOException e) {
3436
throw new ClientException("Failed to read block", e);
3537
}
3638
}
3739

40+
public NativeFormatReader(InputStream inputStream, QuerySettings settings,
41+
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) {
42+
this(inputStream, settings, byteBufferAllocator, NO_TYPE_HINT_MAPPING);
43+
}
44+
3845
@Override
3946
public boolean readRecord(Map<String, Object> record) throws IOException {
4047
if (blockRowIndex >= currentBlock.getnRows()) {

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

Lines changed: 14 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -5,6 +5,7 @@
55
import com.clickhouse.client.api.metadata.TableSchema;
66
import com.clickhouse.client.api.query.QuerySettings;
77
import com.clickhouse.data.ClickHouseColumn;
8+
import com.clickhouse.data.ClickHouseDataType;
89

910
import java.io.EOFException;
1011
import java.io.IOException;
@@ -13,12 +14,22 @@
1314

1415
public class RowBinaryFormatReader extends AbstractBinaryFormatReader {
1516

16-
public RowBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema,
17-
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) {
18-
super(inputStream, querySettings, schema, byteBufferAllocator);
17+
public RowBinaryFormatReader(InputStream inputStream,
18+
QuerySettings querySettings,
19+
TableSchema schema,
20+
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator,
21+
Map<ClickHouseDataType, Class<?>> typeHintMapping) {
22+
super(inputStream, querySettings, schema, byteBufferAllocator, typeHintMapping);
1923
readNextRecord();
2024
}
2125

26+
public RowBinaryFormatReader(InputStream inputStream,
27+
QuerySettings querySettings,
28+
TableSchema schema,
29+
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) {
30+
this(inputStream, querySettings, schema, byteBufferAllocator, NO_TYPE_HINT_MAPPING);
31+
}
32+
2233
@Override
2334
public boolean readRecord(Map<String, Object> record) throws IOException {
2435
boolean firstColumn = true;

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

Lines changed: 13 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,7 @@
66
import com.clickhouse.client.api.metadata.TableSchema;
77
import com.clickhouse.client.api.query.QuerySettings;
88
import com.clickhouse.data.ClickHouseColumn;
9+
import com.clickhouse.data.ClickHouseDataType;
910

1011
import java.io.EOFException;
1112
import java.io.IOException;
@@ -17,12 +18,21 @@
1718

1819
public class RowBinaryWithNamesAndTypesFormatReader extends AbstractBinaryFormatReader implements Iterator<Map<String, Object>> {
1920

20-
public RowBinaryWithNamesAndTypesFormatReader(InputStream inputStream, QuerySettings querySettings,
21-
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) {
22-
super(inputStream, querySettings, null, byteBufferAllocator);
21+
public RowBinaryWithNamesAndTypesFormatReader(InputStream inputStream,
22+
QuerySettings querySettings,
23+
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator,
24+
Map<ClickHouseDataType, Class<?>> typeHintMapping) {
25+
super(inputStream, querySettings, null, byteBufferAllocator, typeHintMapping);
2326
readSchema();
2427
}
2528

29+
public RowBinaryWithNamesAndTypesFormatReader(InputStream inputStream,
30+
QuerySettings querySettings,
31+
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) {
32+
this(inputStream, querySettings, byteBufferAllocator, NO_TYPE_HINT_MAPPING);
33+
}
34+
35+
2636
private void readSchema() {
2737
try {
2838
List<String> names = new ArrayList<>();

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

Lines changed: 15 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -4,21 +4,26 @@
44
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
55
import com.clickhouse.client.api.metadata.TableSchema;
66
import com.clickhouse.client.api.query.QuerySettings;
7+
import com.clickhouse.data.ClickHouseDataType;
78

89
import java.io.EOFException;
910
import java.io.IOException;
1011
import java.io.InputStream;
1112
import java.util.ArrayList;
1213
import java.util.Collections;
1314
import java.util.List;
15+
import java.util.Map;
1416

1517
public class RowBinaryWithNamesFormatReader extends AbstractBinaryFormatReader {
1618

1719
private List<String> columns = null;
1820

19-
public RowBinaryWithNamesFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema,
20-
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) {
21-
super(inputStream, querySettings, schema, byteBufferAllocator);
21+
public RowBinaryWithNamesFormatReader(InputStream inputStream,
22+
QuerySettings querySettings,
23+
TableSchema schema,
24+
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator,
25+
Map<ClickHouseDataType, Class<?>> typeHintMapping) {
26+
super(inputStream, querySettings, schema, byteBufferAllocator, typeHintMapping);
2227
int nCol = 0;
2328
try {
2429
nCol = BinaryStreamReader.readVarInt(input);
@@ -44,6 +49,13 @@ public RowBinaryWithNamesFormatReader(InputStream inputStream, QuerySettings que
4449
readNextRecord();
4550
}
4651

52+
public RowBinaryWithNamesFormatReader(InputStream inputStream,
53+
QuerySettings querySettings,
54+
TableSchema schema,
55+
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) {
56+
this(inputStream, querySettings, schema, byteBufferAllocator, NO_TYPE_HINT_MAPPING);
57+
}
58+
4759
public List<String> getColumns() {
4860
return columns;
4961
}

0 commit comments

Comments
 (0)