diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java index a52a86611..da97c0ecb 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -1,5 +1,14 @@ package com.clickhouse.data; +import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoPointValue; +import com.clickhouse.data.value.ClickHouseGeoPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoRingValue; +import com.clickhouse.data.value.UnsignedByte; +import com.clickhouse.data.value.UnsignedInteger; +import com.clickhouse.data.value.UnsignedLong; +import com.clickhouse.data.value.UnsignedShort; + import java.math.BigDecimal; import java.math.BigInteger; import java.net.Inet4Address; @@ -9,7 +18,6 @@ import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; -import java.time.LocalTime; import java.time.Period; import java.time.ZonedDateTime; import java.time.temporal.ChronoUnit; @@ -29,15 +37,6 @@ import java.util.UUID; import java.util.stream.Collectors; -import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; -import com.clickhouse.data.value.ClickHouseGeoPointValue; -import com.clickhouse.data.value.ClickHouseGeoPolygonValue; -import com.clickhouse.data.value.ClickHouseGeoRingValue; -import com.clickhouse.data.value.UnsignedByte; -import com.clickhouse.data.value.UnsignedInteger; -import com.clickhouse.data.value.UnsignedLong; -import com.clickhouse.data.value.UnsignedShort; - /** * Basic ClickHouse data types. * @@ -101,12 +100,12 @@ public enum ClickHouseDataType implements SQLType { IPv4(Inet4Address.class, false, true, false, 4, 10, 0, 0, 0, false, 0x28, "INET4"), IPv6(Inet6Address.class, false, true, false, 16, 39, 0, 0, 0, false, 0x29, "INET6"), UUID(UUID.class, false, true, false, 16, 69, 0, 0, 0, false, 0x1D), - Point(Object.class, false, true, true, 33, 0, 0, 0, 0, true, 0x2C), // same as Tuple(Float64, Float64) - Polygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Ring) - MultiPolygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Polygon) - Ring(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Point) - LineString( Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Point) - MultiLineString(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Ring) + Point(Object.class, false, true, true, 33, 0, 0, 0, 0, true), // same as Tuple(Float64, Float64) + Polygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Ring) + MultiPolygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Polygon) + Ring(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Point) + LineString( Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Point) + MultiLineString(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Ring) JSON(Object.class, false, false, false, 0, 0, 0, 0, 0, true, 0x30), @Deprecated @@ -372,11 +371,11 @@ public byte getTag() { allAliases = Collections.unmodifiableSet(set); name2type = Collections.unmodifiableMap(map); - Map tmpbinTag2Type = new HashMap<>(); + Map tmpBinTag2Type = new HashMap<>(); for (ClickHouseDataType type : ClickHouseDataType.values()) { - tmpbinTag2Type.put((byte) type.getBinTag(), type); + tmpBinTag2Type.put(type.getBinTag(), type); } - binTag2Type = Collections.unmodifiableMap(tmpbinTag2Type); + binTag2Type = Collections.unmodifiableMap(tmpBinTag2Type); Map tmpIntervalKind2Type = new HashMap<>(); Map tmpIntervalType2Kind = new HashMap<>(); diff --git a/client-v2/pom.xml b/client-v2/pom.xml index f5b7749c1..f178105e3 100644 --- a/client-v2/pom.xml +++ b/client-v2/pom.xml @@ -89,7 +89,7 @@ com.fasterxml.jackson.core jackson-databind test - 2.17.2 + ${jackson.version} ${project.parent.groupId} 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 601e479ca..7e291a11c 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 @@ -25,15 +25,11 @@ import java.time.ZonedDateTime; import java.time.temporal.TemporalAmount; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.Set; import java.util.TimeZone; import java.util.UUID; @@ -58,6 +54,10 @@ public class BinaryStreamReader { private final Class arrayDefaultTypeHint; + private static final int SB_INIT_SIZE = 100; + + private ClickHouseColumn lastDataColumn = null; + /** * Createa a BinaryStreamReader instance that will use the provided buffer allocator. * @@ -110,6 +110,7 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce } ClickHouseColumn actualColumn = column.getDataType() == ClickHouseDataType.Dynamic ? readDynamicData() : column; + lastDataColumn = actualColumn; ClickHouseDataType dataType = actualColumn.getDataType(); int precision = actualColumn.getPrecision(); int scale = actualColumn.getScale(); @@ -1149,73 +1150,176 @@ public byte[] allocate(int size) { } } - private static final Set DECIMAL_TAGS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList( - ClickHouseDataType.Decimal.getBinTag(), - ClickHouseDataType.Decimal32.getBinTag(), - ClickHouseDataType.Decimal64.getBinTag(), - ClickHouseDataType.Decimal128.getBinTag(), - ClickHouseDataType.Decimal256.getBinTag() - ))); - private ClickHouseColumn readDynamicData() throws IOException { byte tag = readByte(); - ClickHouseDataType type; - if (tag == ClickHouseDataType.INTERVAL_BIN_TAG) { - byte intervalKind = readByte(); - type = ClickHouseDataType.intervalKind2Type.get(intervalKind); - if (type == null) { - throw new ClientException("Unsupported interval kind: " + intervalKind); + ClickHouseDataType type = ClickHouseDataType.binTag2Type.get(tag); + if (type == null) { + if (tag == ClickHouseDataType.DateTime64.getBinTag() - 1) { + // without timezone + byte scale = readByte(); + return ClickHouseColumn.of("v", "DateTime64(" + scale + ")"); + } else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) { + String typeName = readString(input); + return ClickHouseColumn.of("v", typeName); + } else if (tag == ClickHouseDataType.TUPLE_WITH_NAMES_BIN_TAG || tag == ClickHouseDataType.TUPLE_WITHOUT_NAMES_BIN_TAG) { + int size = readVarInt(input); + StringBuilder typeNameBuilder = new StringBuilder(SB_INIT_SIZE); + typeNameBuilder.append("Tuple("); + final boolean readName = tag == ClickHouseDataType.TUPLE_WITH_NAMES_BIN_TAG; + for (int i = 0; i < size; i++) { + if (readName) { + String name = readString(input); + typeNameBuilder.append(name).append(' '); + } + ClickHouseColumn column = readDynamicData(); + typeNameBuilder.append(column.getOriginalTypeName()).append(','); + } + typeNameBuilder.setLength(typeNameBuilder.length() - 1); + typeNameBuilder.append(")"); + return ClickHouseColumn.of("v", typeNameBuilder.toString()); + } else { + throw new ClientException("Unsupported data type with tag " + tag); } - return ClickHouseColumn.of("v", type, false, 0, 0); - } else if (tag == ClickHouseDataType.DateTime32.getBinTag()) { - byte scale = readByte(); - return ClickHouseColumn.of("v", "DateTime32(" + scale + ")"); - } else if (tag == ClickHouseDataType.DateTime64.getBinTag() - 1) { // without timezone - byte scale = readByte(); - return ClickHouseColumn.of("v", "DateTime64(" + scale +")"); - } else if (tag == ClickHouseDataType.DateTime64.getBinTag()) { - byte scale = readByte(); - String timezone = readString(input); - return ClickHouseColumn.of("v", "DateTime64(" + scale + (timezone.isEmpty() ? "" : ", " + timezone) +")"); - } else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) { - String typeName = readString(input); - return ClickHouseColumn.of("v", typeName); - } else if (DECIMAL_TAGS.contains(tag)) { - int precision = readByte(); - int scale = readByte(); - return ClickHouseColumn.of("v", ClickHouseDataType.binTag2Type.get(tag), false, precision, scale); - } else if (tag == ClickHouseDataType.Array.getBinTag()) { - ClickHouseColumn elementColumn = readDynamicData(); - return ClickHouseColumn.of("v", "Array(" + elementColumn.getOriginalTypeName() + ")"); - } else if (tag == ClickHouseDataType.Map.getBinTag()) { - ClickHouseColumn keyInfo = readDynamicData(); - ClickHouseColumn valueInfo = readDynamicData(); - return ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + "," + valueInfo.getOriginalTypeName() + ")"); - } else if (tag == ClickHouseDataType.Enum8.getBinTag() || tag == ClickHouseDataType.Enum16.getBinTag()) { - int constants = readVarInt(input); - int[] values = new int[constants]; - String[] names = new String[constants]; - ClickHouseDataType enumType = constants > 127 ? ClickHouseDataType.Enum16 : ClickHouseDataType.Enum8; - for (int i = 0; i < constants; i++) { - names[i] = readString(input); - if (enumType == ClickHouseDataType.Enum8) { - values[i] = readUnsignedByte(); - } else { - values[i] = readUnsignedShortLE(); + } + switch (type) { + case Array: { + ClickHouseColumn elementColumn = readDynamicData(); + return ClickHouseColumn.of("v", "Array(" + elementColumn.getOriginalTypeName() + ")"); + } + case DateTime32: { + String timezone = readString(input); + return ClickHouseColumn.of("v", "DateTime32(" + timezone + ")"); + } + case DateTime64: { + byte scale = readByte(); + String timezone = readString(input); + return ClickHouseColumn.of("v", "DateTime64(" + scale + (timezone.isEmpty() ? "" : ", " + timezone) +")"); + } + case Decimal: + case Decimal32: + case Decimal64: + case Decimal128: + case Decimal256: { + int precision = readByte(); + int scale = readByte(); + return ClickHouseColumn.of("v", ClickHouseDataType.binTag2Type.get(tag), false, precision, scale); + } + case Dynamic: { + int maxTypes = readVarInt(input); + return ClickHouseColumn.of("v", "Dynamic(" + maxTypes + ")"); + } + case Enum: + case Enum8: + case Enum16: { + int constants = readVarInt(input); + int[] values = new int[constants]; + String[] names = new String[constants]; + ClickHouseDataType enumType = constants > 127 ? ClickHouseDataType.Enum16 : ClickHouseDataType.Enum8; + for (int i = 0; i < constants; i++) { + names[i] = readString(input); + if (enumType == ClickHouseDataType.Enum8) { + values[i] = readUnsignedByte(); + } else { + values[i] = readUnsignedShortLE(); + } } + return new ClickHouseColumn(enumType, "v", enumType.name(), false, false, Collections.emptyList(), Collections.emptyList(), + new ClickHouseEnum(names, values)); } - return new ClickHouseColumn(enumType, "v", enumType.name(), false, false, Collections.emptyList(), Collections.emptyList(), - new ClickHouseEnum(names, values)); - } else if (tag == ClickHouseDataType.NULLABLE_BIN_TAG) { - ClickHouseColumn column = readDynamicData(); - return ClickHouseColumn.of("v", "Nullable(" + column.getOriginalTypeName() + ")"); - } else { - type = ClickHouseDataType.binTag2Type.get(tag); - if (type == null) { - throw new ClientException("Unsupported data type with tag " + tag); + case FixedString: { + int length = readVarInt(input); + return ClickHouseColumn.of("v", "FixedString(" + length + ")"); } - return ClickHouseColumn.of("v", type, false, 0, 0); + case IntervalHour: + case IntervalMinute: + case IntervalSecond: + case IntervalDay: + case IntervalMonth: + case IntervalMicrosecond: + case IntervalMillisecond: + case IntervalNanosecond: + case IntervalQuarter: + case IntervalYear: + case IntervalWeek: { + byte intervalKind = readByte(); + type = ClickHouseDataType.intervalKind2Type.get(intervalKind); + if (type == null) { + throw new ClientException("Unsupported interval kind: " + intervalKind); + } + return ClickHouseColumn.of("v", type, false, 0, 0); + } + case JSON: { + byte serializationVersion = readByte(); + int maxDynamicPaths = readVarInt(input); + byte maxDynamicTypes = readByte(); + int numberOfTypedPaths = readVarInt(input); + StringBuilder typeDef = new StringBuilder(SB_INIT_SIZE); + typeDef.append("JSON(max_dynamic_paths=").append(maxDynamicPaths).append(",max_dynamic_types=").append(maxDynamicTypes).append(","); + for (int i = 0; i < numberOfTypedPaths; i++) { + typeDef.append(readString(input)).append(' '); // path + ClickHouseColumn column = readDynamicData(); + typeDef.append(column.getOriginalTypeName()).append(','); + } + int numberOfSkipPaths = readVarInt(input); + for (int i = 0; i < numberOfSkipPaths; i++) { + typeDef.append(readString(input)).append(','); + } + int numberOfPathRegexp = readVarInt(input); + for (int i = 0; i < numberOfPathRegexp; i++) { + typeDef.append(readString(input)).append(','); + } + typeDef.setLength(typeDef.length() - 1); + typeDef.append(')'); + return ClickHouseColumn.of("v", typeDef.toString()); + } + case LowCardinality: { + ClickHouseColumn column = readDynamicData(); + return ClickHouseColumn.of("v", "LowCardinality(" + column.getOriginalTypeName() + ")"); + } + case Map: { + ClickHouseColumn keyInfo = readDynamicData(); + ClickHouseColumn valueInfo = readDynamicData(); + return ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + "," + valueInfo.getOriginalTypeName() + ")"); + } + case Nested: { + int size = readVarInt(input); + StringBuilder nested = new StringBuilder(SB_INIT_SIZE); + nested.append("Nested("); + for (int i = 0; i < size; i++) { + String name = readString(input); + nested.append(name).append(','); + } + nested.setLength(nested.length() - 1); + nested.append(')'); + return ClickHouseColumn.of("v", nested.toString()); + } + case Nullable: { + ClickHouseColumn column = readDynamicData(); + return ClickHouseColumn.of("v", "Nullable(" + column.getOriginalTypeName() + ")"); + } + case Time64: { + byte precision = readByte(); + return ClickHouseColumn.of("v", "Time64(" + precision + ")"); + } + case Variant: { + int variants = readVarInt(input); + StringBuilder variant = new StringBuilder(SB_INIT_SIZE); + variant.append("Variant("); + for (int i = 0; i < variants; i++) { + ClickHouseColumn column = readDynamicData(); + variant.append(column.getOriginalTypeName()).append(','); + } + variant.setLength(variant.length() - 1); + variant.append(")"); + return ClickHouseColumn.of("v", "Variant(" + variant + ")"); + } + case AggregateFunction: + throw new ClientException("Aggregate functions are not supported yet"); + case BFloat16: + throw new ClientException("BFloat16 is not supported yet"); + default: + return ClickHouseColumn.of("v", type, false, 0, 0); } } @@ -1235,6 +1339,9 @@ private Map readJsonData(InputStream input, ClickHouseColumn col ClickHouseColumn dataColumn = predefinedColumns == null? JSON_PLACEHOLDER_COL : predefinedColumns.getOrDefault(path, JSON_PLACEHOLDER_COL); Object value = readValue(dataColumn); + if (value == null && (lastDataColumn != null && lastDataColumn.getDataType() == ClickHouseDataType.Nothing) ) { + continue; + } obj.put(path, value); } return obj; diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/SerializerUtils.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/SerializerUtils.java index 2f2887217..ca1c3714d 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/SerializerUtils.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/SerializerUtils.java @@ -17,6 +17,7 @@ import org.objectweb.asm.MethodVisitor; import org.objectweb.asm.Opcodes; import org.objectweb.asm.Type; + import java.io.IOException; import java.io.OutputStream; import java.lang.reflect.Array; @@ -292,10 +293,6 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ throws IOException { ClickHouseDataType dt = typeColumn.getDataType(); - byte binTag = dt.getBinTag(); - if (binTag == -1) { - throw new ClientException("Type " + dt.name() +" serialization is not supported for Dynamic column"); - } if (typeColumn.isNullable()) { stream.write(ClickHouseDataType.NULLABLE_BIN_TAG); @@ -304,10 +301,24 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ stream.write(ClickHouseDataType.LOW_CARDINALITY_BIN_TAG); } + byte binTag = dt.getBinTag(); + if (binTag == -1) { + switch (dt) { + case Point: + case Polygon: + case Ring: + case MultiPolygon: + stream.write(ClickHouseDataType.CUSTOM_TYPE_BIN_TAG); + BinaryStreamUtils.writeString(stream, dt.name()); + return; + default: + throw new ClientException("Type " + dt.name() +" serialization is not supported for Dynamic column"); + } + } switch (dt) { case FixedString: stream.write(binTag); - writeVarInt(stream, typeColumn.getEstimatedLength()); + BinaryStreamUtils.writeVarInt(stream, typeColumn.getEstimatedLength()); break; case Enum8: case Enum16: @@ -315,7 +326,7 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ ClickHouseEnum enumVal = typeColumn.getEnumConstants(); String[] names = enumVal.getNames(); int[] values = enumVal.getValues(); - writeVarInt(stream, names.length); + BinaryStreamUtils.writeVarInt(stream, names.length); for (int i = 0; i < enumVal.size(); i++ ) { BinaryStreamUtils.writeString(stream, names[i]); if (dt == ClickHouseDataType.Enum8) { @@ -380,13 +391,6 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ // 0x20... stream.write(0x20); break; - case Point: - case Polygon: - case Ring: - case MultiPolygon: - stream.write(ClickHouseDataType.CUSTOM_TYPE_BIN_TAG); - BinaryStreamUtils.writeString(stream, dt.name()); - break; case Variant: stream.write(binTag); break; @@ -659,7 +663,7 @@ private static void serializeTime64(OutputStream stream, Object value) throws IO } else if (value instanceof Long) { BinaryStreamUtils.writeUnsignedInt64(stream, (Long) value); } else if (value instanceof Instant) { - BinaryStreamUtils.writeUnsignedInt64(stream, BigInteger.valueOf(((Instant) value).getEpochSecond()).shiftLeft(32) + BinaryStreamUtils.writeUnsignedInt64(stream, BigInteger.valueOf(((Instant) value).getEpochSecond() * 1_000_000_000L) .add(BigInteger.valueOf(((Instant) value).getNano()))); } else { throw new UnsupportedOperationException("Cannot convert " + value.getClass() + " to Time64"); diff --git a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java index 08c45118a..d8fd279fd 100644 --- a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java @@ -8,12 +8,14 @@ import com.clickhouse.client.api.DataTypeUtils; import com.clickhouse.client.api.command.CommandSettings; import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; +import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader; import com.clickhouse.client.api.data_formats.internal.SerializerUtils; import com.clickhouse.client.api.enums.Protocol; import com.clickhouse.client.api.insert.InsertSettings; import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.GenericRecord; import com.clickhouse.client.api.query.QueryResponse; +import com.clickhouse.client.api.query.QuerySettings; import com.clickhouse.client.api.sql.SQLUtils; import com.clickhouse.data.ClickHouseDataType; import com.clickhouse.data.ClickHouseVersion; @@ -652,18 +654,130 @@ public void testDynamicWithTime64Types() throws Exception { String.valueOf(_999_hours), }); - Instant maxTime64 = Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59, - 999999999); + Instant time64 = Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59); + long time64Value = time64.getEpochSecond() * 1_000_000_000 + time64.getNano(); + testDynamicWith("Time64", + new Object[]{ + time64Value + }, + new String[]{ + String.valueOf(time64Value) + } + ); + Instant maxTime64 = Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59, + 123456789); + long maxTime64Value = maxTime64.getEpochSecond() * 1_000_000_000 + maxTime64.getNano(); testDynamicWith("Time64", new Object[]{ maxTime64, }, new String[]{ - "3958241016481971977" + String.valueOf(maxTime64Value) }); } + @Test(groups = {"integration"}) + public void testDynamicWithNestedTypes() throws Exception { + if (isVersionMatch("(,24.8]")) { + return; + } + + List records = client.queryAll("SELECT (1, 'row1', 0.1)::Tuple(rowId Int32, name String, value Float64)::Dynamic AS row, 10::Int32 AS num"); + + Object[] tuple = (Object[]) records.get(0).getObject("row"); + Assert.assertEquals(tuple[0], 1); + Assert.assertEquals(tuple[1], "row1"); + Assert.assertEquals(tuple[2], 0.1); + Assert.assertEquals(records.get(0).getInteger("num"), 10); + } + + @Test(groups = {"integration"}) + public void testDynamicWithFixedString() throws Exception { + if (isVersionMatch("(,24.8]")) { + return; + } + List records = client.queryAll("SELECT 'row1'::FixedString(4)::Dynamic AS str, 10::Int32 AS num"); + Assert.assertEquals("row1", records.get(0).getString("str")); + Assert.assertEquals(records.get(0).getInteger("num"), 10); // added to check if reading further is not affected + } + + @Test(groups = {"integration"}, dataProvider = "testDynamicWithJSON_dp") + public void testDynamicWithJSON(String type, String json, Object expected) throws Exception { + if (isVersionMatch("(,24.8]")) { + return; + } + List records = client.queryAll("SELECT '" + json + "'::" + type + "::Dynamic AS val"); + GenericRecord row = records.get(0); + Object val = row.getObject("val"); + Assert.assertEquals(val, expected); + } + + @DataProvider + public Object[][] testDynamicWithJSON_dp() { + Map map1 = new HashMap<>(); + map1.put("name", "row1"); + map1.put("value", 0.1); + Map map2 = new HashMap<>(); + map2.put("name", "row1"); + map2.put("value", 0.1f); + Map map3 = new HashMap<>(); + map3.put("a.b", "c"); + map3.put("a.d", "e"); + Map map4 = new HashMap<>(); + map4.put("a.d", "e"); + + return new Object[][] { + { "JSON(max_dynamic_paths=100, max_dynamic_types=100)", "{\"name\": \"row1\", \"value\": 0.1}", map1}, + { "JSON(value Float32)", "{\"name\": \"row1\", \"value\": 0.1}", map2}, + { "JSON", "{ \"a\" : { \"b\" : \"c\", \"d\" : \"e\" } }", map3}, + { "JSON(SKIP a.b)", "{ \"a\" : { \"b\" : \"c\", \"d\" : \"e\" } }", map4}, + { "JSON(SKIP REGEXP \'a\\.b\')", "{ \"a\" : { \"b\" : \"c\", \"d\" : \"e\" } }", map4}, + + }; + } + + @Test(groups = {"integration"}) + public void testDynamicWithJSONWithArrays() throws Exception { + if (isVersionMatch("(,24.8]")) { + return; + } + + String json = "{ \"array\": [ {\"a\": 100 }, {\"b\": \"name\"}]}"; + String type = "JSON(max_dynamic_paths=100, max_dynamic_types=100)"; + List records = client.queryAll("SELECT '" + json + "'::" + type + "::Dynamic AS val"); + GenericRecord row = records.get(0); + HashMap val = (HashMap) row.getObject("val"); + BinaryStreamReader.ArrayValue array = (BinaryStreamReader.ArrayValue) val.get("array"); + List> items = array.asList(); + + Assert.assertEquals(items.size(), 2); + Assert.assertEquals(items.get(0).get("a"), 100L); + Assert.assertEquals(items.get(1).get("b"), "name"); + + } + + @Test(groups = {"integration"}) + public void testDynamicWithVariant() throws Exception { + if (isVersionMatch("(,24.8]")) { + return; + } + + List records = client.queryAll("select arrayJoin([1, 'a', 3]::Array(Variant(String, Int32)))::Dynamic as val"); + + GenericRecord row = records.get(0); + Object val = row.getObject("val"); + Assert.assertEquals(val, 1); + + row = records.get(1); + val = row.getObject("val"); + Assert.assertEquals(val, "a"); + + row = records.get(2); + val = row.getObject("val"); + Assert.assertEquals(val, 3); + } + @Data @AllArgsConstructor public static class DTOForDynamicPrimitivesTests { diff --git a/jdbc-v2/pom.xml b/jdbc-v2/pom.xml index afccb2dd3..708de6eb0 100644 --- a/jdbc-v2/pom.xml +++ b/jdbc-v2/pom.xml @@ -51,6 +51,13 @@ + + com.fasterxml.jackson.core + jackson-databind + test + ${jackson.version} + + ${project.parent.groupId} clickhouse-client 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 dc14eebdb..0f71de053 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java @@ -7,11 +7,16 @@ import com.clickhouse.data.ClickHouseDataType; import com.clickhouse.data.ClickHouseVersion; import com.clickhouse.data.Tuple; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; +import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; import java.net.Inet4Address; @@ -617,7 +622,7 @@ public void testTimeTypes() throws SQLException { Properties createProperties = new Properties(); createProperties.put(ClientConfigProperties.serverSetting("allow_experimental_time_time64_type"), "1"); runQuery("CREATE TABLE test_time64 (order Int8, " - + "time Time('UTC'), time64 Time64(9) " + + "time Time, time64 Time64(9) " + ") ENGINE = MergeTree ORDER BY ()", createProperties); @@ -1960,4 +1965,98 @@ public void testGeoMultiPolygon() throws Exception { } } + private static final HashMap EMPTY_JSON = new HashMap<>(); + + @Test(groups = { "integration" }, dataProvider = "testJSONReadDP") + public void testJSONRead(String json, Object expected) throws Exception { + if (ClickHouseVersion.of(getServerVersion()).check("(,24.8]")) { + return; // JSON was introduced in 24.10 + } + Properties createProperties = new Properties(); + createProperties.put(ClientConfigProperties.serverSetting("allow_experimental_json_type"), "1"); + runQuery("DROP TABLE IF EXISTS test_jdbc_json_read"); + runQuery("CREATE TABLE test_jdbc_json_read (data JSON) ENGINE = MergeTree ORDER BY ()", createProperties); + + try (Connection conn = getJdbcConnection(); Statement stmt = conn.createStatement()) { + final String sql = "INSERT INTO test_jdbc_json_read (data) VALUES ('%s'), ('{}')"; + stmt.executeUpdate(String.format(sql, json)); + + try (ResultSet rs = stmt.executeQuery("SELECT * FROM test_jdbc_json_read")) { + + assertTrue(rs.next()); + Object jsonObj = rs.getObject(1); + if (expected == null) { + expected = jsonToClientMap(json); + } + assertEquals(jsonObj, expected); + assertTrue(rs.next()); + Object emptyJsonObj = rs.getObject(1); + assertEquals(emptyJsonObj, EMPTY_JSON); + assertFalse(rs.next()); + } + } + } + + private final ObjectMapper objectMapper = new ObjectMapper() + .enable(DeserializationFeature.USE_LONG_FOR_INTS); + + private HashMap jsonToClientMap(String json) { + try { + return objectMapper.readValue(json, new TypeReference>() {}); + } catch (IOException e) { + throw new RuntimeException("Failed to read json to Map", e); + } + } + + @DataProvider(name = "testJSONReadDP") + public Object[][] testJSONReadDP() { + Map map1 = new HashMap<>(); + map1.put("nested.key", "value"); + Map map2 = new HashMap<>(); + map2.put("nested.numbers",new ArrayList() {{ add(1L); add(2L); add(3L); }}); + Map map3 = new HashMap<>(); + map3.put("nested.strings", new ArrayList() {{ add("one"); add("two"); add("three"); }}); + Map map4 = new HashMap<>(); + map4.put("array", new ArrayList>() {{ + add(new HashMap() {{ + put("nested.key", "value"); + }}); + add(new HashMap() {{ + put("nested.numbers", new ArrayList() {{ + add(1L); + add(2L); + add(3L); + }}); + }}); + }}); + Map map5 = new HashMap<>(); + map5.put("array", new ArrayList>() {{ + add(new HashMap() {{ + put("nested.strings", new ArrayList() {{ add("one"); add("two"); add("three"); }}); + + }}); + }}); + Map map6 = new HashMap<>(); + map6.put("level1.level2.level3", "value"); + + Map map7 = new HashMap<>(); + map7.put("level1.level2.level3.level4", "value"); + + return new Object[][] { + {"{\"key\": \"value\"}", null}, // Simple object + {"{\"numbers\":[1, 2, 3]}", null}, + {"{\"strings\":[\"one\", \"two\", \"three\"]}", null}, + {"{\"nested\":{\"key\": \"value\"}}", map1}, // nested objects + {"{\"nested\":{\"numbers\":[1, 2, 3]}}", map2}, // nested objects + {"{\"nested\":{\"strings\":[\"one\", \"two\", \"three\"]}}", map3}, // nested objects + {"{\"array\":[{\"key\": \"value\"},{\"key\": \"value\"}]}", null}, // array of objects + {"{\"array\":[{\"numbers\":[1, 2, 3]},{\"strings\":[\"one\", \"two\", \"three\"]}]}", null}, // array of objects + {"{\"array\":[{\"nested\":{\"key\": \"value\"}},{\"nested\":{\"numbers\":[1, 2, 3]}}]}", map4}, // array of objects + {"{\"array\":[{\"nested\":{\"strings\":[\"one\", \"two\", \"three\"]}}]}", map5}, // array of objects + {"{\"array\":[{\"nested\":[{\"key\": \"value\"}]}]}", null}, // simple array of objects + {"{\"level1\": {\"level2\": {\"level3\": \"value\"}}}", map6}, // deep nested objects + {"{\"level1\": {\"level2\": {\"level3\": {\"level4\": \"value\"}}}}", map7}, // deep nested objects + + }; + } } diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java index 5728bc318..85d76b218 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java @@ -49,17 +49,17 @@ protected static String getDatabase() { @Override protected boolean runQuery(String query) { - return runQuery(query, new Properties()); + runQuery(query, new Properties()); + return true; } - protected boolean runQuery(String query, Properties connProperties) { + protected void runQuery(String query, Properties connProperties) { try (Connection connection = getJdbcConnection(connProperties)) { try (Statement stmt = connection.createStatement()) { - return stmt.execute(query); + stmt.execute(query); } } catch (SQLException e) { - LOGGER.error("Failed to run query: {}", query, e); - return false; + throw new RuntimeException(e); } } diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/PreparedStatementTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/PreparedStatementTest.java index 8d8f3e2fe..96a1bb1af 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/PreparedStatementTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/PreparedStatementTest.java @@ -717,10 +717,20 @@ void testMetabaseBug01() throws Exception { stmt.executeBatch(); } + StringBuilder sb = new StringBuilder(); + try (Statement stmt01 = conn.createStatement()) { + try (ResultSet rs = stmt01.executeQuery("SELECT id, name FROM `users_tmp`")) { + while (rs.next()) { + sb.append(rs.getInt(1)).append(",").append(rs.getString(2)).append(";"); + } + } + } + + try (Statement stmt01 = conn.createStatement()) { try (ResultSet rs = stmt01.executeQuery("SELECT count(*) FROM `users_tmp`")) { assertTrue(rs.next()); - assertEquals(rs.getInt(1), 3); + assertEquals(rs.getInt(1), 3,"Users in users_tmp: " + sb); } } diff --git a/pom.xml b/pom.xml index e3e208dc0..1625e9821 100644 --- a/pom.xml +++ b/pom.xml @@ -137,6 +137,7 @@ 1.37 33.4.6-jre 0.8.0 + 2.17.2 17 17