Skip to content

Commit 6a4127c

Browse files
authored
Merge pull request #2601 from ClickHouse/fix_json_issues_093
[Client-V2, JDBC] Fix problem reading JSON nested arrays
2 parents 2ea587c + b55ab7e commit 6a4127c

File tree

10 files changed

+449
-108
lines changed

10 files changed

+449
-108
lines changed

clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java

Lines changed: 18 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,14 @@
11
package com.clickhouse.data;
22

3+
import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue;
4+
import com.clickhouse.data.value.ClickHouseGeoPointValue;
5+
import com.clickhouse.data.value.ClickHouseGeoPolygonValue;
6+
import com.clickhouse.data.value.ClickHouseGeoRingValue;
7+
import com.clickhouse.data.value.UnsignedByte;
8+
import com.clickhouse.data.value.UnsignedInteger;
9+
import com.clickhouse.data.value.UnsignedLong;
10+
import com.clickhouse.data.value.UnsignedShort;
11+
312
import java.math.BigDecimal;
413
import java.math.BigInteger;
514
import java.net.Inet4Address;
@@ -9,7 +18,6 @@
918
import java.time.Instant;
1019
import java.time.LocalDate;
1120
import java.time.LocalDateTime;
12-
import java.time.LocalTime;
1321
import java.time.Period;
1422
import java.time.ZonedDateTime;
1523
import java.time.temporal.ChronoUnit;
@@ -29,15 +37,6 @@
2937
import java.util.UUID;
3038
import java.util.stream.Collectors;
3139

32-
import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue;
33-
import com.clickhouse.data.value.ClickHouseGeoPointValue;
34-
import com.clickhouse.data.value.ClickHouseGeoPolygonValue;
35-
import com.clickhouse.data.value.ClickHouseGeoRingValue;
36-
import com.clickhouse.data.value.UnsignedByte;
37-
import com.clickhouse.data.value.UnsignedInteger;
38-
import com.clickhouse.data.value.UnsignedLong;
39-
import com.clickhouse.data.value.UnsignedShort;
40-
4140
/**
4241
* Basic ClickHouse data types.
4342
*
@@ -101,12 +100,12 @@ public enum ClickHouseDataType implements SQLType {
101100
IPv4(Inet4Address.class, false, true, false, 4, 10, 0, 0, 0, false, 0x28, "INET4"),
102101
IPv6(Inet6Address.class, false, true, false, 16, 39, 0, 0, 0, false, 0x29, "INET6"),
103102
UUID(UUID.class, false, true, false, 16, 69, 0, 0, 0, false, 0x1D),
104-
Point(Object.class, false, true, true, 33, 0, 0, 0, 0, true, 0x2C), // same as Tuple(Float64, Float64)
105-
Polygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Ring)
106-
MultiPolygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Polygon)
107-
Ring(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Point)
108-
LineString( Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Point)
109-
MultiLineString(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Ring)
103+
Point(Object.class, false, true, true, 33, 0, 0, 0, 0, true), // same as Tuple(Float64, Float64)
104+
Polygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Ring)
105+
MultiPolygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Polygon)
106+
Ring(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Point)
107+
LineString( Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Point)
108+
MultiLineString(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Ring)
110109

111110
JSON(Object.class, false, false, false, 0, 0, 0, 0, 0, true, 0x30),
112111
@Deprecated
@@ -372,11 +371,11 @@ public byte getTag() {
372371
allAliases = Collections.unmodifiableSet(set);
373372
name2type = Collections.unmodifiableMap(map);
374373

375-
Map<Byte, ClickHouseDataType> tmpbinTag2Type = new HashMap<>();
374+
Map<Byte, ClickHouseDataType> tmpBinTag2Type = new HashMap<>();
376375
for (ClickHouseDataType type : ClickHouseDataType.values()) {
377-
tmpbinTag2Type.put((byte) type.getBinTag(), type);
376+
tmpBinTag2Type.put(type.getBinTag(), type);
378377
}
379-
binTag2Type = Collections.unmodifiableMap(tmpbinTag2Type);
378+
binTag2Type = Collections.unmodifiableMap(tmpBinTag2Type);
380379

381380
Map<Byte, ClickHouseDataType> tmpIntervalKind2Type = new HashMap<>();
382381
Map<ClickHouseDataType, ClickHouseDataType.IntervalKind > tmpIntervalType2Kind = new HashMap<>();

client-v2/pom.xml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -89,7 +89,7 @@
8989
<groupId>com.fasterxml.jackson.core</groupId>
9090
<artifactId>jackson-databind</artifactId>
9191
<scope>test</scope>
92-
<version>2.17.2</version>
92+
<version>${jackson.version}</version>
9393
</dependency>
9494
<dependency>
9595
<groupId>${project.parent.groupId}</groupId>

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

Lines changed: 171 additions & 64 deletions
Original file line numberDiff line numberDiff line change
@@ -25,15 +25,11 @@
2525
import java.time.ZonedDateTime;
2626
import java.time.temporal.TemporalAmount;
2727
import java.util.ArrayList;
28-
import java.util.Arrays;
2928
import java.util.Collections;
3029
import java.util.HashMap;
31-
import java.util.HashSet;
3230
import java.util.LinkedHashMap;
3331
import java.util.List;
3432
import java.util.Map;
35-
import java.util.Objects;
36-
import java.util.Set;
3733
import java.util.TimeZone;
3834
import java.util.UUID;
3935

@@ -58,6 +54,10 @@ public class BinaryStreamReader {
5854

5955
private final Class<?> arrayDefaultTypeHint;
6056

57+
private static final int SB_INIT_SIZE = 100;
58+
59+
private ClickHouseColumn lastDataColumn = null;
60+
6161
/**
6262
* Createa a BinaryStreamReader instance that will use the provided buffer allocator.
6363
*
@@ -110,6 +110,7 @@ public <T> T readValue(ClickHouseColumn column, Class<?> typeHint) throws IOExce
110110
}
111111

112112
ClickHouseColumn actualColumn = column.getDataType() == ClickHouseDataType.Dynamic ? readDynamicData() : column;
113+
lastDataColumn = actualColumn;
113114
ClickHouseDataType dataType = actualColumn.getDataType();
114115
int precision = actualColumn.getPrecision();
115116
int scale = actualColumn.getScale();
@@ -1149,73 +1150,176 @@ public byte[] allocate(int size) {
11491150
}
11501151
}
11511152

1152-
private static final Set<Byte> DECIMAL_TAGS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
1153-
ClickHouseDataType.Decimal.getBinTag(),
1154-
ClickHouseDataType.Decimal32.getBinTag(),
1155-
ClickHouseDataType.Decimal64.getBinTag(),
1156-
ClickHouseDataType.Decimal128.getBinTag(),
1157-
ClickHouseDataType.Decimal256.getBinTag()
1158-
)));
1159-
11601153
private ClickHouseColumn readDynamicData() throws IOException {
11611154
byte tag = readByte();
11621155

1163-
ClickHouseDataType type;
1164-
if (tag == ClickHouseDataType.INTERVAL_BIN_TAG) {
1165-
byte intervalKind = readByte();
1166-
type = ClickHouseDataType.intervalKind2Type.get(intervalKind);
1167-
if (type == null) {
1168-
throw new ClientException("Unsupported interval kind: " + intervalKind);
1156+
ClickHouseDataType type = ClickHouseDataType.binTag2Type.get(tag);
1157+
if (type == null) {
1158+
if (tag == ClickHouseDataType.DateTime64.getBinTag() - 1) {
1159+
// without timezone
1160+
byte scale = readByte();
1161+
return ClickHouseColumn.of("v", "DateTime64(" + scale + ")");
1162+
} else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) {
1163+
String typeName = readString(input);
1164+
return ClickHouseColumn.of("v", typeName);
1165+
} else if (tag == ClickHouseDataType.TUPLE_WITH_NAMES_BIN_TAG || tag == ClickHouseDataType.TUPLE_WITHOUT_NAMES_BIN_TAG) {
1166+
int size = readVarInt(input);
1167+
StringBuilder typeNameBuilder = new StringBuilder(SB_INIT_SIZE);
1168+
typeNameBuilder.append("Tuple(");
1169+
final boolean readName = tag == ClickHouseDataType.TUPLE_WITH_NAMES_BIN_TAG;
1170+
for (int i = 0; i < size; i++) {
1171+
if (readName) {
1172+
String name = readString(input);
1173+
typeNameBuilder.append(name).append(' ');
1174+
}
1175+
ClickHouseColumn column = readDynamicData();
1176+
typeNameBuilder.append(column.getOriginalTypeName()).append(',');
1177+
}
1178+
typeNameBuilder.setLength(typeNameBuilder.length() - 1);
1179+
typeNameBuilder.append(")");
1180+
return ClickHouseColumn.of("v", typeNameBuilder.toString());
1181+
} else {
1182+
throw new ClientException("Unsupported data type with tag " + tag);
11691183
}
1170-
return ClickHouseColumn.of("v", type, false, 0, 0);
1171-
} else if (tag == ClickHouseDataType.DateTime32.getBinTag()) {
1172-
byte scale = readByte();
1173-
return ClickHouseColumn.of("v", "DateTime32(" + scale + ")");
1174-
} else if (tag == ClickHouseDataType.DateTime64.getBinTag() - 1) { // without timezone
1175-
byte scale = readByte();
1176-
return ClickHouseColumn.of("v", "DateTime64(" + scale +")");
1177-
} else if (tag == ClickHouseDataType.DateTime64.getBinTag()) {
1178-
byte scale = readByte();
1179-
String timezone = readString(input);
1180-
return ClickHouseColumn.of("v", "DateTime64(" + scale + (timezone.isEmpty() ? "" : ", " + timezone) +")");
1181-
} else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) {
1182-
String typeName = readString(input);
1183-
return ClickHouseColumn.of("v", typeName);
1184-
} else if (DECIMAL_TAGS.contains(tag)) {
1185-
int precision = readByte();
1186-
int scale = readByte();
1187-
return ClickHouseColumn.of("v", ClickHouseDataType.binTag2Type.get(tag), false, precision, scale);
1188-
} else if (tag == ClickHouseDataType.Array.getBinTag()) {
1189-
ClickHouseColumn elementColumn = readDynamicData();
1190-
return ClickHouseColumn.of("v", "Array(" + elementColumn.getOriginalTypeName() + ")");
1191-
} else if (tag == ClickHouseDataType.Map.getBinTag()) {
1192-
ClickHouseColumn keyInfo = readDynamicData();
1193-
ClickHouseColumn valueInfo = readDynamicData();
1194-
return ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + "," + valueInfo.getOriginalTypeName() + ")");
1195-
} else if (tag == ClickHouseDataType.Enum8.getBinTag() || tag == ClickHouseDataType.Enum16.getBinTag()) {
1196-
int constants = readVarInt(input);
1197-
int[] values = new int[constants];
1198-
String[] names = new String[constants];
1199-
ClickHouseDataType enumType = constants > 127 ? ClickHouseDataType.Enum16 : ClickHouseDataType.Enum8;
1200-
for (int i = 0; i < constants; i++) {
1201-
names[i] = readString(input);
1202-
if (enumType == ClickHouseDataType.Enum8) {
1203-
values[i] = readUnsignedByte();
1204-
} else {
1205-
values[i] = readUnsignedShortLE();
1184+
}
1185+
switch (type) {
1186+
case Array: {
1187+
ClickHouseColumn elementColumn = readDynamicData();
1188+
return ClickHouseColumn.of("v", "Array(" + elementColumn.getOriginalTypeName() + ")");
1189+
}
1190+
case DateTime32: {
1191+
String timezone = readString(input);
1192+
return ClickHouseColumn.of("v", "DateTime32(" + timezone + ")");
1193+
}
1194+
case DateTime64: {
1195+
byte scale = readByte();
1196+
String timezone = readString(input);
1197+
return ClickHouseColumn.of("v", "DateTime64(" + scale + (timezone.isEmpty() ? "" : ", " + timezone) +")");
1198+
}
1199+
case Decimal:
1200+
case Decimal32:
1201+
case Decimal64:
1202+
case Decimal128:
1203+
case Decimal256: {
1204+
int precision = readByte();
1205+
int scale = readByte();
1206+
return ClickHouseColumn.of("v", ClickHouseDataType.binTag2Type.get(tag), false, precision, scale);
1207+
}
1208+
case Dynamic: {
1209+
int maxTypes = readVarInt(input);
1210+
return ClickHouseColumn.of("v", "Dynamic(" + maxTypes + ")");
1211+
}
1212+
case Enum:
1213+
case Enum8:
1214+
case Enum16: {
1215+
int constants = readVarInt(input);
1216+
int[] values = new int[constants];
1217+
String[] names = new String[constants];
1218+
ClickHouseDataType enumType = constants > 127 ? ClickHouseDataType.Enum16 : ClickHouseDataType.Enum8;
1219+
for (int i = 0; i < constants; i++) {
1220+
names[i] = readString(input);
1221+
if (enumType == ClickHouseDataType.Enum8) {
1222+
values[i] = readUnsignedByte();
1223+
} else {
1224+
values[i] = readUnsignedShortLE();
1225+
}
12061226
}
1227+
return new ClickHouseColumn(enumType, "v", enumType.name(), false, false, Collections.emptyList(), Collections.emptyList(),
1228+
new ClickHouseEnum(names, values));
12071229
}
1208-
return new ClickHouseColumn(enumType, "v", enumType.name(), false, false, Collections.emptyList(), Collections.emptyList(),
1209-
new ClickHouseEnum(names, values));
1210-
} else if (tag == ClickHouseDataType.NULLABLE_BIN_TAG) {
1211-
ClickHouseColumn column = readDynamicData();
1212-
return ClickHouseColumn.of("v", "Nullable(" + column.getOriginalTypeName() + ")");
1213-
} else {
1214-
type = ClickHouseDataType.binTag2Type.get(tag);
1215-
if (type == null) {
1216-
throw new ClientException("Unsupported data type with tag " + tag);
1230+
case FixedString: {
1231+
int length = readVarInt(input);
1232+
return ClickHouseColumn.of("v", "FixedString(" + length + ")");
12171233
}
1218-
return ClickHouseColumn.of("v", type, false, 0, 0);
1234+
case IntervalHour:
1235+
case IntervalMinute:
1236+
case IntervalSecond:
1237+
case IntervalDay:
1238+
case IntervalMonth:
1239+
case IntervalMicrosecond:
1240+
case IntervalMillisecond:
1241+
case IntervalNanosecond:
1242+
case IntervalQuarter:
1243+
case IntervalYear:
1244+
case IntervalWeek: {
1245+
byte intervalKind = readByte();
1246+
type = ClickHouseDataType.intervalKind2Type.get(intervalKind);
1247+
if (type == null) {
1248+
throw new ClientException("Unsupported interval kind: " + intervalKind);
1249+
}
1250+
return ClickHouseColumn.of("v", type, false, 0, 0);
1251+
}
1252+
case JSON: {
1253+
byte serializationVersion = readByte();
1254+
int maxDynamicPaths = readVarInt(input);
1255+
byte maxDynamicTypes = readByte();
1256+
int numberOfTypedPaths = readVarInt(input);
1257+
StringBuilder typeDef = new StringBuilder(SB_INIT_SIZE);
1258+
typeDef.append("JSON(max_dynamic_paths=").append(maxDynamicPaths).append(",max_dynamic_types=").append(maxDynamicTypes).append(",");
1259+
for (int i = 0; i < numberOfTypedPaths; i++) {
1260+
typeDef.append(readString(input)).append(' '); // path
1261+
ClickHouseColumn column = readDynamicData();
1262+
typeDef.append(column.getOriginalTypeName()).append(',');
1263+
}
1264+
int numberOfSkipPaths = readVarInt(input);
1265+
for (int i = 0; i < numberOfSkipPaths; i++) {
1266+
typeDef.append(readString(input)).append(',');
1267+
}
1268+
int numberOfPathRegexp = readVarInt(input);
1269+
for (int i = 0; i < numberOfPathRegexp; i++) {
1270+
typeDef.append(readString(input)).append(',');
1271+
}
1272+
typeDef.setLength(typeDef.length() - 1);
1273+
typeDef.append(')');
1274+
return ClickHouseColumn.of("v", typeDef.toString());
1275+
}
1276+
case LowCardinality: {
1277+
ClickHouseColumn column = readDynamicData();
1278+
return ClickHouseColumn.of("v", "LowCardinality(" + column.getOriginalTypeName() + ")");
1279+
}
1280+
case Map: {
1281+
ClickHouseColumn keyInfo = readDynamicData();
1282+
ClickHouseColumn valueInfo = readDynamicData();
1283+
return ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + "," + valueInfo.getOriginalTypeName() + ")");
1284+
}
1285+
case Nested: {
1286+
int size = readVarInt(input);
1287+
StringBuilder nested = new StringBuilder(SB_INIT_SIZE);
1288+
nested.append("Nested(");
1289+
for (int i = 0; i < size; i++) {
1290+
String name = readString(input);
1291+
nested.append(name).append(',');
1292+
}
1293+
nested.setLength(nested.length() - 1);
1294+
nested.append(')');
1295+
return ClickHouseColumn.of("v", nested.toString());
1296+
}
1297+
case Nullable: {
1298+
ClickHouseColumn column = readDynamicData();
1299+
return ClickHouseColumn.of("v", "Nullable(" + column.getOriginalTypeName() + ")");
1300+
}
1301+
case Time64: {
1302+
byte precision = readByte();
1303+
return ClickHouseColumn.of("v", "Time64(" + precision + ")");
1304+
}
1305+
case Variant: {
1306+
int variants = readVarInt(input);
1307+
StringBuilder variant = new StringBuilder(SB_INIT_SIZE);
1308+
variant.append("Variant(");
1309+
for (int i = 0; i < variants; i++) {
1310+
ClickHouseColumn column = readDynamicData();
1311+
variant.append(column.getOriginalTypeName()).append(',');
1312+
}
1313+
variant.setLength(variant.length() - 1);
1314+
variant.append(")");
1315+
return ClickHouseColumn.of("v", "Variant(" + variant + ")");
1316+
}
1317+
case AggregateFunction:
1318+
throw new ClientException("Aggregate functions are not supported yet");
1319+
case BFloat16:
1320+
throw new ClientException("BFloat16 is not supported yet");
1321+
default:
1322+
return ClickHouseColumn.of("v", type, false, 0, 0);
12191323
}
12201324
}
12211325

@@ -1235,6 +1339,9 @@ private Map<String, Object> readJsonData(InputStream input, ClickHouseColumn col
12351339
ClickHouseColumn dataColumn = predefinedColumns == null? JSON_PLACEHOLDER_COL :
12361340
predefinedColumns.getOrDefault(path, JSON_PLACEHOLDER_COL);
12371341
Object value = readValue(dataColumn);
1342+
if (value == null && (lastDataColumn != null && lastDataColumn.getDataType() == ClickHouseDataType.Nothing) ) {
1343+
continue;
1344+
}
12381345
obj.put(path, value);
12391346
}
12401347
return obj;

0 commit comments

Comments
 (0)