Skip to content

Commit aa857ed

Browse files
committed
Implemeted configuration for default type hint
1 parent 5c99223 commit aa857ed

File tree

12 files changed

+277
-23
lines changed

12 files changed

+277
-23
lines changed

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

Lines changed: 43 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;
@@ -191,6 +196,26 @@ private Client(Set<String> endpoints, Map<String,String> configuration,
191196
}
192197

193198
this.serverVersion = configuration.getOrDefault(ClientConfigProperties.SERVER_VERSION.getKey(), "unknown");
199+
200+
this.typeHintMapping = translateTypeHintMapping(configuration.get(ClientConfigProperties.TYPE_HINT_MAPPING.getKey()));
201+
}
202+
203+
private Map<ClickHouseDataType, Class<?>> translateTypeHintMapping(String mappingStr) {
204+
if (mappingStr == null || mappingStr.isEmpty()) {
205+
return AbstractBinaryFormatReader.NO_TYPE_HINT_MAPPING;
206+
}
207+
208+
Map<String, String> mapping= ClientConfigProperties.toKeyValuePairs(mappingStr);
209+
Map<ClickHouseDataType, Class<?>> hintMapping = new HashMap<>();
210+
try {
211+
for (Map.Entry<String, String> entry : mapping.entrySet()) {
212+
hintMapping.put(ClickHouseDataType.of(entry.getKey()),
213+
Class.forName(entry.getValue()));
214+
}
215+
} catch (ClassNotFoundException e) {
216+
throw new ClientMisconfigurationException("Failed to translate type-hint mapping", e);
217+
}
218+
return hintMapping;
194219
}
195220

196221
/**
@@ -1003,6 +1028,20 @@ public Builder setServerVersion(String serverVersion) {
10031028
return this;
10041029
}
10051030

1031+
/**
1032+
* Defines mapping between ClickHouse data type and target Java type
1033+
* Used by binary readers to convert values into desired Java type.
1034+
* @param typeHintMapping - map between ClickHouse data type and Java class
1035+
* @return this builder instance
1036+
*/
1037+
public Builder typeHintMapping(Map<ClickHouseDataType, Class<?>> typeHintMapping) {
1038+
this.configuration.put(ClientConfigProperties.TYPE_HINT_MAPPING.getKey(),
1039+
ClientConfigProperties.mapToString(typeHintMapping, (v) -> {
1040+
return ((Class<?>) v).getName();
1041+
}));
1042+
return this;
1043+
}
1044+
10061045
public Client build() {
10071046
// check if endpoint are empty. so can not initiate client
10081047
if (this.endpoints.isEmpty()) {
@@ -1921,23 +1960,20 @@ public ClickHouseBinaryFormatReader newBinaryFormatReader(QueryResponse response
19211960
BinaryStreamReader.ByteBufferAllocator byteBufferPool = useCachingBufferAllocator ?
19221961
new BinaryStreamReader.CachingByteBufferAllocator() :
19231962
new BinaryStreamReader.DefaultByteBufferAllocator();
1924-
19251963
switch (response.getFormat()) {
19261964
case Native:
19271965
reader = new NativeFormatReader(response.getInputStream(), response.getSettings(),
1928-
byteBufferPool);
1966+
byteBufferPool, typeHintMapping);
19291967
break;
19301968
case RowBinaryWithNamesAndTypes:
1931-
reader = new RowBinaryWithNamesAndTypesFormatReader(response.getInputStream(), response.getSettings(),
1932-
byteBufferPool);
1969+
reader = new RowBinaryWithNamesAndTypesFormatReader(response.getInputStream(), response.getSettings(), byteBufferPool, typeHintMapping);
19331970
break;
19341971
case RowBinaryWithNames:
1935-
reader = new RowBinaryWithNamesFormatReader(response.getInputStream(), response.getSettings(), schema,
1936-
byteBufferPool);
1972+
reader = new RowBinaryWithNamesFormatReader(response.getInputStream(), response.getSettings(), schema, byteBufferPool, typeHintMapping);
19371973
break;
19381974
case RowBinary:
19391975
reader = new RowBinaryFormatReader(response.getInputStream(), response.getSettings(), schema,
1940-
byteBufferPool);
1976+
byteBufferPool, typeHintMapping);
19411977
break;
19421978
default:
19431979
throw new IllegalArgumentException("Binary readers doesn't support format: " + response.getFormat());

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

Lines changed: 74 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5,8 +5,11 @@
55
import java.util.Arrays;
66
import java.util.Collection;
77
import java.util.Collections;
8+
import java.util.LinkedHashMap;
89
import java.util.List;
910
import java.util.Locale;
11+
import java.util.Map;
12+
import java.util.function.Function;
1013
import java.util.stream.Collectors;
1114

1215
/**
@@ -140,6 +143,12 @@ public enum ClientConfigProperties {
140143
* Name of the group under which client metrics appear
141144
*/
142145
METRICS_GROUP_NAME("metrics_name"),
146+
147+
/**
148+
* Defines mapping between ClickHouse data type and target Java type
149+
* Used by binary readers to convert values into desired Java type.
150+
*/
151+
TYPE_HINT_MAPPING("type_hint_mapping"),
143152
;
144153

145154
private final String key;
@@ -207,4 +216,69 @@ public static List<String> valuesFromCommaSeparated(String value) {
207216
return Arrays.stream(value.split("(?<!\\\\),")).map(s -> s.replaceAll("\\\\,", ","))
208217
.collect(Collectors.toList());
209218
}
219+
220+
221+
/**
222+
* Converts given string to key value pairs.
223+
*
224+
* @param str string
225+
* @return non-null key value pairs
226+
*/
227+
public static Map<String, String> toKeyValuePairs(String str) {
228+
if (str == null || str.isEmpty()) {
229+
return Collections.emptyMap();
230+
}
231+
232+
Map<String, String> map = new LinkedHashMap<>();
233+
String key = null;
234+
StringBuilder builder = new StringBuilder();
235+
for (int i = 0, len = str.length(); i < len; i++) {
236+
char ch = str.charAt(i);
237+
if (ch == '\\' && i + 1 < len) {
238+
ch = str.charAt(++i);
239+
builder.append(ch);
240+
continue;
241+
}
242+
243+
if (Character.isWhitespace(ch)) {
244+
if (builder.length() > 0) {
245+
builder.append(ch);
246+
}
247+
} else if (ch == '=' && key == null) {
248+
key = builder.toString().trim();
249+
builder.setLength(0);
250+
} else if (ch == ',' && key != null) {
251+
String value = builder.toString().trim();
252+
builder.setLength(0);
253+
if (!key.isEmpty() && !value.isEmpty()) {
254+
map.put(key, value);
255+
}
256+
key = null;
257+
} else {
258+
builder.append(ch);
259+
}
260+
}
261+
262+
if (key != null && builder.length() > 0) {
263+
String value = builder.toString().trim();
264+
if (!key.isEmpty() && !value.isEmpty()) {
265+
map.put(key, value);
266+
}
267+
}
268+
269+
return Collections.unmodifiableMap(map);
270+
}
271+
272+
273+
public static String mapToString(Map<?,?> map, Function<Object, String> valueConverter) {
274+
StringBuilder sb = new StringBuilder();
275+
for (Map.Entry<?, ?> entry : map.entrySet()) {
276+
sb.append(entry.getKey()).append("=").append(valueConverter.apply(entry.getValue())).append(",");
277+
}
278+
279+
if (sb.length() > 0) {
280+
sb.setLength(sb.length() - 1);
281+
}
282+
return sb.toString();
283+
}
210284
}

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
}

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

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -51,6 +51,8 @@
5151

5252
public abstract class AbstractBinaryFormatReader implements ClickHouseBinaryFormatReader {
5353

54+
public static final Map<ClickHouseDataType, Class<?>> NO_TYPE_HINT_MAPPING = null;
55+
5456
private static final Logger LOG = LoggerFactory.getLogger(AbstractBinaryFormatReader.class);
5557

5658
protected InputStream input;
@@ -63,8 +65,7 @@ public abstract class AbstractBinaryFormatReader implements ClickHouseBinaryForm
6365
private boolean hasNext = true;
6466
private boolean initialState = true; // reader is in initial state, no records have been read yet
6567

66-
protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema,
67-
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) {
68+
protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema,BinaryStreamReader.ByteBufferAllocator byteBufferAllocator, Map<ClickHouseDataType, Class<?>> defaultTypeHintMap) {
6869
this.input = inputStream;
6970
Map<String, Object> settings = querySettings == null ? Collections.emptyMap() : querySettings.getAllSettings();
7071
Boolean useServerTimeZone = (Boolean) settings.get(ClientConfigProperties.USE_SERVER_TIMEZONE.getKey());
@@ -75,7 +76,8 @@ protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings quer
7576
}
7677
boolean jsonAsString = MapUtils.getFlag(settings,
7778
ClientConfigProperties.serverSetting(ServerSettings.OUTPUT_FORMAT_BINARY_WRITE_JSON_AS_STRING), false);
78-
this.binaryStreamReader = new BinaryStreamReader(inputStream, timeZone, LOG, byteBufferAllocator, jsonAsString);
79+
this.binaryStreamReader = new BinaryStreamReader(inputStream, timeZone, LOG, byteBufferAllocator, jsonAsString,
80+
defaultTypeHintMap);
7981
if (schema != null) {
8082
setSchema(schema);
8183
}

0 commit comments

Comments
 (0)