Skip to content

Commit 028be17

Browse files
committed
Added JSON tests
1 parent e4e8402 commit 028be17

File tree

4 files changed

+162
-57
lines changed

4 files changed

+162
-57
lines changed

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

Lines changed: 15 additions & 16 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

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

Lines changed: 27 additions & 28 deletions
Original file line numberDiff line numberDiff line change
@@ -32,11 +32,9 @@
3232
import java.util.LinkedHashMap;
3333
import java.util.List;
3434
import java.util.Map;
35-
import java.util.Objects;
3635
import java.util.Set;
3736
import java.util.TimeZone;
3837
import java.util.UUID;
39-
import java.util.stream.Collectors;
4038

4139
/**
4240
* This class is not thread safe and should not be shared between multiple threads.
@@ -1163,7 +1161,32 @@ private ClickHouseColumn readDynamicData() throws IOException {
11631161

11641162
ClickHouseDataType type = ClickHouseDataType.binTag2Type.get(tag);
11651163
if (type == null) {
1166-
throw new ClientException("Unsupported data type with tag " + tag);
1164+
if (tag == ClickHouseDataType.DateTime64.getBinTag() - 1) {
1165+
// without timezone
1166+
byte scale = readByte();
1167+
return ClickHouseColumn.of("v", "DateTime64(" + scale + ")");
1168+
} else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) {
1169+
String typeName = readString(input);
1170+
return ClickHouseColumn.of("v", typeName);
1171+
} else if (tag == ClickHouseDataType.TUPLE_WITH_NAMES_BIN_TAG || tag == ClickHouseDataType.TUPLE_WITHOUT_NAMES_BIN_TAG) {
1172+
int size = readVarInt(input);
1173+
StringBuilder typeNameBuilder = new StringBuilder();
1174+
typeNameBuilder.append("Tuple(");
1175+
final boolean readName = tag == ClickHouseDataType.TUPLE_WITH_NAMES_BIN_TAG;
1176+
for (int i = 0; i < size; i++) {
1177+
if (readName) {
1178+
String name = readString(input);
1179+
typeNameBuilder.append(name).append(' ');
1180+
}
1181+
ClickHouseColumn column = readDynamicData();
1182+
typeNameBuilder.append(column.getOriginalTypeName()).append(',');
1183+
}
1184+
typeNameBuilder.setLength(typeNameBuilder.length() - 1);
1185+
typeNameBuilder.append(")");
1186+
return ClickHouseColumn.of("v", typeNameBuilder.toString());
1187+
} else {
1188+
throw new ClientException("Unsupported data type with tag " + tag);
1189+
}
11671190
}
11681191
switch (type) {
11691192
case Array: {
@@ -1240,7 +1263,7 @@ private ClickHouseColumn readDynamicData() throws IOException {
12401263
StringBuilder typeDef = new StringBuilder();
12411264
typeDef.append("JSON(max_dynamic_paths=").append(maxDynamicPaths).append(",max_dynamic_types=").append(maxDynamicTypes).append(",");
12421265
for (int i = 0; i < numberOfTypedPaths; i++) {
1243-
typeDef.append(readString(input)); // path
1266+
typeDef.append(readString(input)).append(' '); // path
12441267
ClickHouseColumn column = readDynamicData();
12451268
typeDef.append(column.getOriginalTypeName()).append(',');
12461269
}
@@ -1302,30 +1325,6 @@ private ClickHouseColumn readDynamicData() throws IOException {
13021325
case BFloat16:
13031326
throw new ClientException("BFloat16 is not supported yet");
13041327
default:
1305-
if (tag == ClickHouseDataType.DateTime64.getBinTag() - 1) {
1306-
// without timezone
1307-
byte scale = readByte();
1308-
return ClickHouseColumn.of("v", "DateTime64(" + scale + ")");
1309-
} else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) {
1310-
String typeName = readString(input);
1311-
return ClickHouseColumn.of("v", typeName);
1312-
} else if (tag == ClickHouseDataType.TUPLE_WITH_NAMES_BIN_TAG || tag == ClickHouseDataType.TUPLE_WITHOUT_NAMES_BIN_TAG) {
1313-
int size = readVarInt(input);
1314-
StringBuilder typeNameBuilder = new StringBuilder();
1315-
typeNameBuilder.append("Tuple(");
1316-
final boolean readName = tag == ClickHouseDataType.TUPLE_WITH_NAMES_BIN_TAG;
1317-
for (int i = 0; i < size; i++) {
1318-
if (readName) {
1319-
String name = readString(input);
1320-
typeNameBuilder.append(name).append(' ');
1321-
}
1322-
ClickHouseColumn column = readDynamicData();
1323-
typeNameBuilder.append(column.getOriginalTypeName()).append(',');
1324-
}
1325-
typeNameBuilder.setLength(typeNameBuilder.length() - 1);
1326-
typeNameBuilder.append(")");
1327-
return ClickHouseColumn.of("v", typeNameBuilder.toString());
1328-
}
13291328
return ClickHouseColumn.of("v", type, false, 0, 0);
13301329
}
13311330
}

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

Lines changed: 17 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
import org.objectweb.asm.MethodVisitor;
1818
import org.objectweb.asm.Opcodes;
1919
import org.objectweb.asm.Type;
20+
2021
import java.io.IOException;
2122
import java.io.OutputStream;
2223
import java.lang.reflect.Array;
@@ -292,10 +293,6 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ
292293
throws IOException {
293294

294295
ClickHouseDataType dt = typeColumn.getDataType();
295-
byte binTag = dt.getBinTag();
296-
if (binTag == -1) {
297-
throw new ClientException("Type " + dt.name() +" serialization is not supported for Dynamic column");
298-
}
299296

300297
if (typeColumn.isNullable()) {
301298
stream.write(ClickHouseDataType.NULLABLE_BIN_TAG);
@@ -304,18 +301,32 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ
304301
stream.write(ClickHouseDataType.LOW_CARDINALITY_BIN_TAG);
305302
}
306303

304+
byte binTag = dt.getBinTag();
305+
if (binTag == -1) {
306+
switch (dt) {
307+
case Point:
308+
case Polygon:
309+
case Ring:
310+
case MultiPolygon:
311+
stream.write(ClickHouseDataType.CUSTOM_TYPE_BIN_TAG);
312+
BinaryStreamUtils.writeString(stream, dt.name());
313+
return;
314+
default:
315+
throw new ClientException("Type " + dt.name() +" serialization is not supported for Dynamic column");
316+
}
317+
}
307318
switch (dt) {
308319
case FixedString:
309320
stream.write(binTag);
310-
writeVarInt(stream, typeColumn.getEstimatedLength());
321+
BinaryStreamUtils.writeVarInt(stream, typeColumn.getEstimatedLength());
311322
break;
312323
case Enum8:
313324
case Enum16:
314325
stream.write(binTag);
315326
ClickHouseEnum enumVal = typeColumn.getEnumConstants();
316327
String[] names = enumVal.getNames();
317328
int[] values = enumVal.getValues();
318-
writeVarInt(stream, names.length);
329+
BinaryStreamUtils.writeVarInt(stream, names.length);
319330
for (int i = 0; i < enumVal.size(); i++ ) {
320331
BinaryStreamUtils.writeString(stream, names[i]);
321332
if (dt == ClickHouseDataType.Enum8) {
@@ -380,13 +391,6 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ
380391
// 0x20<var_uint_number_of_elements><var_uint_name_size_1><name_data_1><nested_type_encoding_1>...<var_uint_name_size_N><name_data_N><nested_type_encoding_N>
381392
stream.write(0x20);
382393
break;
383-
case Point:
384-
case Polygon:
385-
case Ring:
386-
case MultiPolygon:
387-
stream.write(ClickHouseDataType.CUSTOM_TYPE_BIN_TAG);
388-
BinaryStreamUtils.writeString(stream, dt.name());
389-
break;
390394
case Variant:
391395
stream.write(binTag);
392396
break;

client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java

Lines changed: 103 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -8,12 +8,14 @@
88
import com.clickhouse.client.api.DataTypeUtils;
99
import com.clickhouse.client.api.command.CommandSettings;
1010
import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
11+
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
1112
import com.clickhouse.client.api.data_formats.internal.SerializerUtils;
1213
import com.clickhouse.client.api.enums.Protocol;
1314
import com.clickhouse.client.api.insert.InsertSettings;
1415
import com.clickhouse.client.api.metadata.TableSchema;
1516
import com.clickhouse.client.api.query.GenericRecord;
1617
import com.clickhouse.client.api.query.QueryResponse;
18+
import com.clickhouse.client.api.query.QuerySettings;
1719
import com.clickhouse.client.api.sql.SQLUtils;
1820
import com.clickhouse.data.ClickHouseDataType;
1921
import com.clickhouse.data.ClickHouseVersion;
@@ -676,6 +678,107 @@ public void testDynamicWithTime64Types() throws Exception {
676678
});
677679
}
678680

681+
@Test(groups = {"integration"})
682+
public void testDynamicWithNestedTypes() throws Exception {
683+
if (isVersionMatch("(,24.8]")) {
684+
return;
685+
}
686+
687+
List<GenericRecord> records = client.queryAll("SELECT (1, 'row1', 0.1)::Tuple(rowId Int32, name String, value Float64)::Dynamic AS row, 10::Int32 AS num");
688+
689+
Object[] tuple = (Object[]) records.get(0).getObject("row");
690+
Assert.assertEquals(tuple[0], 1);
691+
Assert.assertEquals(tuple[1], "row1");
692+
Assert.assertEquals(tuple[2], 0.1);
693+
Assert.assertEquals(records.get(0).getInteger("num"), 10);
694+
}
695+
696+
@Test(groups = {"integration"})
697+
public void testDynamicWithFixedString() throws Exception {
698+
if (isVersionMatch("(,24.8]")) {
699+
return;
700+
}
701+
List<GenericRecord> records = client.queryAll("SELECT 'row1'::FixedString(4)::Dynamic AS str, 10::Int32 AS num");
702+
Assert.assertEquals("row1", records.get(0).getString("str"));
703+
Assert.assertEquals(records.get(0).getInteger("num"), 10); // added to check if reading further is not affected
704+
}
705+
706+
@Test(groups = {"integration"}, dataProvider = "testDynamicWithJSON_dp")
707+
public void testDynamicWithJSON(String type, String json, Object expected) throws Exception {
708+
if (isVersionMatch("(,24.8]")) {
709+
return;
710+
}
711+
List<GenericRecord> records = client.queryAll("SELECT '" + json + "'::" + type + "::Dynamic AS val");
712+
GenericRecord row = records.get(0);
713+
Object val = row.getObject("val");
714+
Assert.assertEquals(val, expected);
715+
}
716+
717+
@DataProvider
718+
public Object[][] testDynamicWithJSON_dp() {
719+
Map<String, Object> map1 = new HashMap<>();
720+
map1.put("name", "row1");
721+
map1.put("value", 0.1);
722+
Map<String, Object> map2 = new HashMap<>();
723+
map2.put("name", "row1");
724+
map2.put("value", 0.1f);
725+
Map<String, Object> map3 = new HashMap<>();
726+
map3.put("a.b", "c");
727+
map3.put("a.d", "e");
728+
Map<String, Object> map4 = new HashMap<>();
729+
map4.put("a.d", "e");
730+
731+
return new Object[][] {
732+
{ "JSON(max_dynamic_paths=100, max_dynamic_types=100)", "{\"name\": \"row1\", \"value\": 0.1}", map1},
733+
{ "JSON(value Float32)", "{\"name\": \"row1\", \"value\": 0.1}", map2},
734+
{ "JSON", "{ \"a\" : { \"b\" : \"c\", \"d\" : \"e\" } }", map3},
735+
{ "JSON(SKIP a.b)", "{ \"a\" : { \"b\" : \"c\", \"d\" : \"e\" } }", map4},
736+
{ "JSON(SKIP REGEXP \'a\\.b\')", "{ \"a\" : { \"b\" : \"c\", \"d\" : \"e\" } }", map4},
737+
738+
};
739+
}
740+
741+
@Test(groups = {"integration"})
742+
public void testDynamicWithJSONWithArrays() throws Exception {
743+
if (isVersionMatch("(,24.8]")) {
744+
return;
745+
}
746+
747+
String json = "{ \"array\": [ {\"a\": 100 }, {\"b\": \"name\"}]}";
748+
String type = "JSON(max_dynamic_paths=100, max_dynamic_types=100)";
749+
List<GenericRecord> records = client.queryAll("SELECT '" + json + "'::" + type + "::Dynamic AS val");
750+
GenericRecord row = records.get(0);
751+
HashMap<String, Object> val = (HashMap<String, Object>) row.getObject("val");
752+
BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) val.get("array");
753+
List<HashMap<String, Object>> items = array.asList();
754+
755+
Assert.assertEquals(items.size(), 2);
756+
Assert.assertEquals(items.get(0).get("a"), 100L);
757+
Assert.assertEquals(items.get(1).get("b"), "name");
758+
759+
}
760+
761+
@Test(groups = {"integration"})
762+
public void testDynamicWithVariant() throws Exception {
763+
if (isVersionMatch("(,24.8]")) {
764+
return;
765+
}
766+
767+
List<GenericRecord> records = client.queryAll("select arrayJoin([1, 'a', 3]::Array(Variant(String, Int32)))::Dynamic as val");
768+
769+
GenericRecord row = records.get(0);
770+
Object val = row.getObject("val");
771+
Assert.assertEquals(val, 1);
772+
773+
row = records.get(1);
774+
val = row.getObject("val");
775+
Assert.assertEquals(val, "a");
776+
777+
row = records.get(2);
778+
val = row.getObject("val");
779+
Assert.assertEquals(val, 3);
780+
}
781+
679782
@Data
680783
@AllArgsConstructor
681784
public static class DTOForDynamicPrimitivesTests {

0 commit comments

Comments
 (0)