From 589a152bf063688bfe6ef891af3d4fc925c59aee Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 5 Dec 2024 09:51:32 -0800 Subject: [PATCH 1/7] draft Variant implementation --- .../com/clickhouse/data/ClickHouseColumn.java | 13 +++- .../clickhouse/data/ClickHouseDataType.java | 4 +- .../format/ClickHouseRowBinaryProcessor.java | 69 +++++++++++++++++++ .../jdbc/ClickHouseStatementTest.java | 28 ++++++++ 4 files changed, 112 insertions(+), 2 deletions(-) diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java index a9acaa37e..5786bad82 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java @@ -41,6 +41,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.LinkedList; import java.util.List; import java.util.Objects; @@ -65,6 +66,7 @@ public final class ClickHouseColumn implements Serializable { private static final String KEYWORD_OBJECT = ClickHouseDataType.Object.name(); private static final String KEYWORD_MAP = ClickHouseDataType.Map.name(); private static final String KEYWORD_NESTED = ClickHouseDataType.Nested.name(); + private static final String KEYWORD_VARIANT = ClickHouseDataType.Variant.name(); private int columnCount; private int columnIndex; @@ -273,6 +275,9 @@ private static ClickHouseColumn update(ClickHouseColumn column) { case Nothing: column.template = ClickHouseEmptyValue.INSTANCE; break; + case Variant: + column.template = ClickHouseTupleValue.of(); + break; default: break; } @@ -398,7 +403,8 @@ protected static int readColumn(String args, int startIndex, int len, String nam fixedLength = false; estimatedLength++; } else if (args.startsWith(matchedKeyword = KEYWORD_TUPLE, i) - || args.startsWith(matchedKeyword = KEYWORD_OBJECT, i)) { + || args.startsWith(matchedKeyword = KEYWORD_OBJECT, i) + || args.startsWith(matchedKeyword = KEYWORD_VARIANT, i)) { int index = args.indexOf('(', i + matchedKeyword.length()); if (index < i) { throw new IllegalArgumentException(ERROR_MISSING_NESTED_TYPE); @@ -410,12 +416,17 @@ protected static int readColumn(String args, int startIndex, int len, String nam if (c == ')') { break; } else if (c != ',' && !Character.isWhitespace(c)) { + String columnName = ""; i = readColumn(args, i, endIndex, "", nestedColumns); } } if (nestedColumns.isEmpty()) { throw new IllegalArgumentException("Tuple should have at least one nested column"); } + if (matchedKeyword.equals(KEYWORD_VARIANT)) { + nestedColumns.sort(Comparator.comparing(o -> o.getDataType().name())); + nestedColumns.forEach(c -> c.columnName = "v." + c.getDataType().name()); + } column = new ClickHouseColumn(ClickHouseDataType.valueOf(matchedKeyword), name, args.substring(startIndex, endIndex + 1), nullable, lowCardinality, null, nestedColumns); for (ClickHouseColumn n : nestedColumns) { 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 9c077f7c0..507bca4a7 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -101,7 +101,9 @@ public enum ClickHouseDataType { Nothing(Object.class, false, true, false, 0, 0, 0, 0, 0, true), SimpleAggregateFunction(String.class, true, true, false, 0, 0, 0, 0, 0, false), // implementation-defined intermediate state - AggregateFunction(String.class, true, true, false, 0, 0, 0, 0, 0, true); + AggregateFunction(String.class, true, true, false, 0, 0, 0, 0, 0, true), + Variant(List.class, true, true, false, 0, 0, 0, 0, 0, true); + /** * Immutable set(sorted) for all aliases. diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/format/ClickHouseRowBinaryProcessor.java b/clickhouse-data/src/main/java/com/clickhouse/data/format/ClickHouseRowBinaryProcessor.java index cd4239874..78c2b9e21 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/format/ClickHouseRowBinaryProcessor.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/format/ClickHouseRowBinaryProcessor.java @@ -245,6 +245,68 @@ public void serialize(ClickHouseValue value, ClickHouseOutputStream output) thro } } + public static class VariantDeserializer extends ClickHouseDeserializer.CompositeDeserializer { + private final ClickHouseValue[] values; + public VariantDeserializer(ClickHouseDataConfig config, ClickHouseColumn column, + ClickHouseDeserializer... deserializers) { + super(deserializers); + + List nestedCols = column.getNestedColumns(); + int len = nestedCols.size(); + if (deserializers.length != len) { + throw new IllegalArgumentException( + ClickHouseUtils.format("Expect %d deserializers but got %d", len, deserializers.length)); + } + values = new ClickHouseValue[len]; + for (int i = 0; i < len; i++) { + values[i] = nestedCols.get(i).newValue(config); + } + } + + @Override + public ClickHouseValue deserialize(ClickHouseValue ref, ClickHouseInputStream input) throws IOException { + int len = values.length; + Object[] tupleValues = new Object[len]; + int ordTypeNum = BinaryStreamUtils.readInt8(input); + for (int i = 0; i < len; i++) { + if (ordTypeNum == i) { + tupleValues[i] = deserializers[i].deserialize(values[i], input).asObject(); + } else { + tupleValues[i] = null; + } + } + return ref.update(tupleValues); + } + } + + public static class VariantSerializer extends ClickHouseSerializer.CompositeSerializer { + private final ClickHouseValue[] values; + + public VariantSerializer(ClickHouseDataConfig config, ClickHouseColumn column, + ClickHouseSerializer... serializers) { + super(serializers); + + List nestedCols = column.getNestedColumns(); + int len = nestedCols.size(); + if (serializers.length != len) { + throw new IllegalArgumentException( + ClickHouseUtils.format("Expect %d serializers but got %d", len, serializers.length)); + } + values = new ClickHouseValue[len]; + for (int i = 0; i < len; i++) { + values[i] = nestedCols.get(i).newValue(config); + } + } + + @Override + public void serialize(ClickHouseValue value, ClickHouseOutputStream output) throws IOException { + List tupleValues = value.asTuple(); + for (int i = 0, len = serializers.length; i < len; i++) { + serializers[i].serialize(values[i].update(tupleValues.get(i)), output); + } + } + } + @Override protected void readAndFill(ClickHouseRecord r) throws IOException { ClickHouseInputStream in = input; @@ -512,6 +574,10 @@ public ClickHouseDeserializer getDeserializer(ClickHouseDataConfig config, Click } deserializer = new BitmapSerDe(config, column)::deserialize; break; + case Variant: + deserializer = new VariantDeserializer(config, column, + getDeserializers(config, column.getNestedColumns())); + break; default: throw new IllegalArgumentException("Unsupported column:" + column.toString()); } @@ -668,6 +734,9 @@ public ClickHouseSerializer getSerializer(ClickHouseDataConfig config, ClickHous } serializer = new BitmapSerDe(config, column)::serialize; break; + case Variant: + serializer = new VariantSerializer(config, column, getSerializers(config, column.getNestedColumns())); + break; default: throw new IllegalArgumentException("Unsupported column:" + column.toString()); } diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java index 9f1ccb117..6f5076a72 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java @@ -1521,4 +1521,32 @@ public void testMaxResultsRows() throws SQLException { "Unexpected exception: " + e.getMessage()); } } + + @Test(groups = "integration") + public void testVariantDataType() throws SQLException { + String table = "test_variant_type_01"; + Properties props = new Properties(); + props.setProperty("custom_settings", "allow_experimental_variant_type=1"); + props.setProperty(ClickHouseClientOption.COMPRESS.getKey(), "false"); + try (ClickHouseConnection conn = newConnection(props); + ClickHouseStatement s = conn.createStatement()) { + + s.execute("DROP TABLE IF EXISTS " + table); + s.execute("CREATE TABLE " + table +" ( id Variant(UInt32, String, UUID), name String) Engine = MergeTree ORDER BY ()"); + + s.execute("insert into " + table + " values ( 1, 'just number' )"); + s.execute("insert into " + table + " values ( 'i-am-id-01', 'ID as string' ) "); + s.execute("insert into " + table + " values ( generateUUIDv4(), 'ID as UUID' ) "); + + try (ResultSet rs = s.executeQuery("SELECT * FROM " + table)) { + while (rs.next()) { + Object variantValue = rs.getObject(1); + Object name = rs.getString(2); + Object variantSubColumn = rs.getObject("v.String"); + System.out.println("-> " + name + " : " + variantValue); + System.out.println("sub: " + variantSubColumn); + } + } + } + } } From 23d6a933ac5b5b64f2f10ffbb94f9b5d509f5bc9 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 8 Jan 2025 17:12:22 -0800 Subject: [PATCH 2/7] draft implementation of Variant in v2 --- .../com/clickhouse/data/ClickHouseColumn.java | 17 ++- .../clickhouse/data/ClickHouseDataType.java | 74 +++++++++++- .../format/ClickHouseRowBinaryProcessor.java | 1 + client-v2/pom.xml | 6 + .../internal/AbstractBinaryFormatReader.java | 1 + .../internal/BinaryStreamReader.java | 7 ++ .../internal/SerializerUtils.java | 35 ++++++ .../client/datatypes/DataTypeTests.java | 114 ++++++++++++++++++ .../client/datatypes/NestedTypesDTO.java | 28 +++++ .../client/datatypes/VariantDTO.java | 22 ++++ .../client/internal/SmallTests.java | 1 + 11 files changed, 304 insertions(+), 2 deletions(-) create mode 100644 client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java create mode 100644 client-v2/src/test/java/com/clickhouse/client/datatypes/NestedTypesDTO.java create mode 100644 client-v2/src/test/java/com/clickhouse/client/datatypes/VariantDTO.java diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java index 5786bad82..48b70ee06 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java @@ -37,13 +37,16 @@ import java.io.Serializable; import java.lang.reflect.Array; +import java.math.BigInteger; import java.time.OffsetDateTime; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.TimeZone; @@ -94,6 +97,8 @@ public final class ClickHouseColumn implements Serializable { private ClickHouseValue template; + private Map, Integer> classToVariantOrdNumMap; + private static ClickHouseColumn update(ClickHouseColumn column) { column.enumConstants = ClickHouseEnum.EMPTY; int size = column.parameters.size(); @@ -423,9 +428,14 @@ protected static int readColumn(String args, int startIndex, int len, String nam if (nestedColumns.isEmpty()) { throw new IllegalArgumentException("Tuple should have at least one nested column"); } + + List variantDataTypes = new ArrayList<>(); if (matchedKeyword.equals(KEYWORD_VARIANT)) { nestedColumns.sort(Comparator.comparing(o -> o.getDataType().name())); - nestedColumns.forEach(c -> c.columnName = "v." + c.getDataType().name()); + nestedColumns.forEach(c -> { + c.columnName = "v." + c.getDataType().name(); + variantDataTypes.add(c.dataType); + }); } column = new ClickHouseColumn(ClickHouseDataType.valueOf(matchedKeyword), name, args.substring(startIndex, endIndex + 1), nullable, lowCardinality, null, nestedColumns); @@ -435,6 +445,7 @@ protected static int readColumn(String args, int startIndex, int len, String nam fixedLength = false; } } + column.classToVariantOrdNumMap = ClickHouseDataType.buildVariantMapping(variantDataTypes); } if (column == null) { @@ -638,6 +649,10 @@ public boolean isAggregateFunction() { } + public int getVariantOrdNum(Object value) { + return classToVariantOrdNumMap.getOrDefault(value.getClass(), -1); + } + public boolean isArray() { return dataType == ClickHouseDataType.Array; } 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 507bca4a7..9fa83e458 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -8,12 +8,16 @@ import java.time.LocalDateTime; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; +import java.util.EnumSet; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.TreeMap; import java.util.TreeSet; import java.util.UUID; @@ -102,7 +106,75 @@ public enum ClickHouseDataType { SimpleAggregateFunction(String.class, true, true, false, 0, 0, 0, 0, 0, false), // implementation-defined intermediate state AggregateFunction(String.class, true, true, false, 0, 0, 0, 0, 0, true), - Variant(List.class, true, true, false, 0, 0, 0, 0, 0, true); + Variant(List.class, true, true, false, 0, 0, 0, 0, 0, true), + + ; + + public static final List ORDERED_BY_RANGE_INT_TYPES = + Collections.unmodifiableList(Arrays.asList( + Int8, UInt8, Int16, UInt16, Int32, UInt32, Int64, UInt64, Int128, UInt128, Int256, UInt256 + )); + + public static final List ORDERED_BY_RANGE_DECIMAL_TYPES = + Collections.unmodifiableList(Arrays.asList( + Float32, Float64, Decimal32, Decimal64, Decimal128, Decimal256, Decimal + )); + + public static Map, Integer> buildVariantMapping(List variantDataTypes) { + Map, Integer> variantMapping = new HashMap<>(); + + TreeMap intTypesMappings = new TreeMap<>(Comparator.comparingInt(ORDERED_BY_RANGE_INT_TYPES::indexOf)); + TreeMap decTypesMappings = new TreeMap<>(Comparator.comparingInt(ORDERED_BY_RANGE_DECIMAL_TYPES::indexOf)); + + for (int ordNum = 0; ordNum < variantDataTypes.size(); ordNum++) { + ClickHouseDataType dataType = variantDataTypes.get(ordNum); + Set> classSet = DATA_TYPE_TO_CLASS.get(dataType); + + final int finalOrdNum = ordNum; + if (classSet != null) { + if (ORDERED_BY_RANGE_INT_TYPES.contains(dataType)) { + intTypesMappings.put(dataType, ordNum); + } else if (ORDERED_BY_RANGE_DECIMAL_TYPES.contains(dataType)) { + decTypesMappings.put(dataType, ordNum); + } else { + classSet.forEach(c -> variantMapping.put(c, finalOrdNum)); + } + } + } + + // add numbers mappings + for (java.util.Map.Entry entry : intTypesMappings.entrySet()) { + DATA_TYPE_TO_CLASS.get(entry.getKey()).forEach(c -> variantMapping.put(c, entry.getValue())); + } + for (java.util.Map.Entry entry : decTypesMappings.entrySet()) { + DATA_TYPE_TO_CLASS.get(entry.getKey()).forEach(c -> variantMapping.put(c, entry.getValue())); + } + + + return variantMapping; + } + + static final Map>> DATA_TYPE_TO_CLASS = dataTypeClassMap(); + static Map>> dataTypeClassMap() { + Map>> map = new HashMap<>(); + + map.put(UInt256, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class, BigInteger.class)))); + map.put(Int256, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class, BigInteger.class)))); + map.put(UInt128, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class, BigInteger.class)))); + map.put(Int128, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class, BigInteger.class)))); + map.put(UInt64, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class)))); + map.put(Int64, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class)))); + map.put(UInt32, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class )))); + map.put(Int32, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class)))); + map.put(UInt16, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class)))); + map.put(Int16, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class)))); + map.put(UInt8, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class)))); + map.put(Int8, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class)))); + + map.put(String, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(String.class)))); + + return map; + } /** diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/format/ClickHouseRowBinaryProcessor.java b/clickhouse-data/src/main/java/com/clickhouse/data/format/ClickHouseRowBinaryProcessor.java index 78c2b9e21..332c062b6 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/format/ClickHouseRowBinaryProcessor.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/format/ClickHouseRowBinaryProcessor.java @@ -301,6 +301,7 @@ public VariantSerializer(ClickHouseDataConfig config, ClickHouseColumn column, @Override public void serialize(ClickHouseValue value, ClickHouseOutputStream output) throws IOException { List tupleValues = value.asTuple(); + // TODO: variant index for (int i = 0, len = serializers.length; i < len; i++) { serializers[i].serialize(values[i].update(tupleValues.get(i)), output); } diff --git a/client-v2/pom.xml b/client-v2/pom.xml index 1a7fa5a52..c89ea7315 100644 --- a/client-v2/pom.xml +++ b/client-v2/pom.xml @@ -135,6 +135,12 @@ test + + org.projectlombok + lombok + 1.18.36 + test + diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java index cab0afa6f..3bca484fc 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java @@ -251,6 +251,7 @@ protected void setSchema(TableSchema schema) { case String: case Enum8: case Enum16: + case Variant: this.convertions[i] = NumberConverter.NUMBER_CONVERTERS; break; default: 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 2aa6671d2..7d5cacc96 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 @@ -214,6 +214,8 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce return (T) readValue(column.getNestedColumns().get(0)); case AggregateFunction: return (T) readBitmap( column); + case Variant: + return (T) readVariant(column); default: throw new IllegalArgumentException("Unsupported data type: " + column.getDataType()); } @@ -675,6 +677,11 @@ public Object[] readTuple(ClickHouseColumn column) throws IOException { return tuple; } + public Object readVariant(ClickHouseColumn column) throws IOException { + int ordNum = readByte(); + return readValue(column.getNestedColumns().get(ordNum)); + } + /** * Reads a GEO point as an array of two doubles what represents coordinates (X, Y). * @return X, Y coordinates 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 fa4015192..4ce4a33de 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 @@ -28,6 +28,7 @@ import java.time.ZoneId; import java.time.ZonedDateTime; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -219,6 +220,15 @@ private static void serializePrimitiveData(OutputStream stream, Object value, Cl case JSON: serializeJSON(stream, value); break; + case Tuple: + serializeTuple(stream, column, (Object[]) value); + break; + case Point: + serializeTuple(stream, GEO_POINT_TUPLE, (Object[]) value); + break; + case Variant: + serializerVariant(stream, column, value); + break; default: throw new UnsupportedOperationException("Unsupported data type: " + column.getDataType()); } @@ -232,6 +242,31 @@ private static void serializeJSON(OutputStream stream, Object value) throws IOEx } } + private static void serializeTuple(OutputStream out, ClickHouseColumn column, Object[] tupleValues) throws IOException { + if (column.getNestedColumns().size() != tupleValues.length) { + throw new IllegalArgumentException("Column " + column.getColumnName() + " defines as Tuple with " + + column.getNestedColumns().size() +" elements, but only " + tupleValues.length + " provided"); + } + + List nested = column.getNestedColumns(); + for (int i = 0; i < nested.size() ; i++) { + serializeData(out, tupleValues[i], nested.get(i)); + } + } + + + private static void serializerVariant(OutputStream out, ClickHouseColumn column, Object value) throws IOException { + int typeOrdNum = column.getVariantOrdNum(value); + if (typeOrdNum != -1) { + BinaryStreamUtils.writeUnsignedInt8(out, typeOrdNum); + serializeData(out, value, column.getNestedColumns().get(typeOrdNum)); + } else { + throw new IllegalArgumentException("Cannot write value of class " + value.getClass() + " into column with variant type " + column.getOriginalTypeName()); + } + } + + private static final ClickHouseColumn GEO_POINT_TUPLE = ClickHouseColumn.parse("geopoint Tuple(Float64, Float64)").get(0); + private static void serializeAggregateFunction(OutputStream stream, Object value, ClickHouseColumn column) throws IOException { if (column.getAggregateFunction() == ClickHouseAggregateFunction.groupBitmap) { if (value == null) { 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 new file mode 100644 index 000000000..57d70ec3b --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java @@ -0,0 +1,114 @@ +package com.clickhouse.client.datatypes; + +import com.clickhouse.client.BaseIntegrationTest; +import com.clickhouse.client.ClickHouseNode; +import com.clickhouse.client.ClickHouseProtocol; +import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.command.CommandSettings; +import com.clickhouse.client.api.enums.Protocol; +import com.clickhouse.client.api.insert.InsertSettings; +import com.clickhouse.client.api.query.GenericRecord; +import com.clickhouse.client.api.query.QueryResponse; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +public class DataTypeTests extends BaseIntegrationTest { + + private Client client; + private InsertSettings settings; + + private boolean useClientCompression = false; + + private boolean useHttpCompression = false; + + private static final int EXECUTE_CMD_TIMEOUT = 10; // seconds + + public DataTypeTests(boolean useClientCompression, boolean useHttpCompression) { + this.useClientCompression = useClientCompression; + this.useHttpCompression = useHttpCompression; + } + + public DataTypeTests() { + this(false, false); + } + + @BeforeMethod(groups = {"integration"}) + public void setUp() throws IOException { + ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); + client = new Client.Builder() + .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), false) + .setUsername("default") + .setPassword("") + .useNewImplementation(System.getProperty("client.tests.useNewImplementation", "true").equals("true")) + .compressClientRequest(useClientCompression) + .useHttpCompression(useHttpCompression) + .build(); + } + + @AfterMethod(groups = { "integration" }) + public void tearDown() { + client.close(); + } + + + @Test + public void testNestedDataTypes() throws Exception { + final String table = "test_nested_types"; + String tblCreateSQL = NestedTypesDTO.tblCreateSQL(table); + client.execute("DROP TABLE IF EXISTS " + table).get(); + client.execute(tblCreateSQL); + + client.register(NestedTypesDTO.class, client.getTableSchema(table)); + + List data = + Arrays.asList(new NestedTypesDTO(0, new Object[] {(short)127, "test 1"}, new Double[] {0.3d, 0.4d} )); + client.insert(table, data); + + List rows = client.queryAll("SELECT * FROM " + table); + for (GenericRecord row : rows) { + NestedTypesDTO dto = data.get(row.getInteger("rowId")); + Assert.assertEquals(row.getTuple("tuple1"), dto.getTuple1()); + Assert.assertEquals(row.getGeoPoint("point1").getValue(), dto.getPoint1()); + } + + } + + @Test + public void testVariantDataTypeWithPrimitives() throws Exception { + final String table = "test_variant_primitives"; + String tblCreateSQL = VariantDTO.tblCreateSQL(table); + client.execute("DROP TABLE IF EXISTS " + table).get(); + client.execute(tblCreateSQL, (CommandSettings) new CommandSettings().serverSetting("enable_variant_type", "1")); + + client.register(VariantDTO.class, client.getTableSchema(table)); + +// List data = Arrays.asList(new VariantDTO(1, (short)200), new VariantDTO(2, (byte)127), new VariantDTO(3, "test ☺")); + List data = Arrays.asList(new VariantDTO(1, (short)200), new VariantDTO(2, (byte)127)); + client.insert(table, data); + + List rows = client.queryAll("SELECT * FROM " + table); + for (GenericRecord row : rows) { + System.out.println(row.getInteger("rowId") + " " + row.getInteger("a")); + } + } + + + public static String tableDefinition(String table, String... columns) { + StringBuilder sb = new StringBuilder(); + sb.append("CREATE TABLE " + table + " ( "); + Arrays.stream(columns).forEach(s -> { + sb.append(s).append(", "); + }); + sb.setLength(sb.length() - 2); + sb.append(") Engine = MergeTree ORDER BY ()"); + return sb.toString(); + } + +} diff --git a/client-v2/src/test/java/com/clickhouse/client/datatypes/NestedTypesDTO.java b/client-v2/src/test/java/com/clickhouse/client/datatypes/NestedTypesDTO.java new file mode 100644 index 000000000..5f5800405 --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/NestedTypesDTO.java @@ -0,0 +1,28 @@ +package com.clickhouse.client.datatypes; + + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import org.testng.annotations.Test; + +import static com.clickhouse.client.datatypes.DataTypeTests.tableDefinition; + +@Data +@NoArgsConstructor +@AllArgsConstructor +public class NestedTypesDTO { + + private int rowId; + + private Object[] tuple1; + + private Object[] point1; + + public static String tblCreateSQL(String table) { + return tableDefinition(table, + "rowId Int16", + "tuple1 Tuple(Int16, String)", + "point1 Point"); + } +} diff --git a/client-v2/src/test/java/com/clickhouse/client/datatypes/VariantDTO.java b/client-v2/src/test/java/com/clickhouse/client/datatypes/VariantDTO.java new file mode 100644 index 000000000..e48be26a6 --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/VariantDTO.java @@ -0,0 +1,22 @@ +package com.clickhouse.client.datatypes; + + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +import static com.clickhouse.client.datatypes.DataTypeTests.tableDefinition; + +@Data +@AllArgsConstructor +@NoArgsConstructor +public class VariantDTO { + + private int rowId; + + private Object a; + + public static String tblCreateSQL(String table) { + return tableDefinition(table, "rowId Int16, a Variant(String, Int16)"); + } +} diff --git a/client-v2/src/test/java/com/clickhouse/client/internal/SmallTests.java b/client-v2/src/test/java/com/clickhouse/client/internal/SmallTests.java index cf7658798..5a1755777 100644 --- a/client-v2/src/test/java/com/clickhouse/client/internal/SmallTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/internal/SmallTests.java @@ -7,6 +7,7 @@ import org.testng.annotations.Test; import java.math.BigDecimal; +import java.math.BigInteger; import java.time.ZoneId; import java.time.ZonedDateTime; From 166ed6e73126e613df5ab92493684f0e4533d35c Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 21 Jan 2025 23:39:03 -0800 Subject: [PATCH 3/7] tested most primitive types --- .../clickhouse/data/ClickHouseDataType.java | 69 ++++- .../internal/SerializerUtils.java | 124 +++++--- .../client/datatypes/DataTypeTests.java | 146 ++++++++- .../datatypes/DataTypesTestingPOJO.java | 283 ++++++++++++++++++ .../client/datatypes/VariantDTO.java | 19 +- .../clickhouse/client/insert/SamplePOJO.java | 4 + 6 files changed, 571 insertions(+), 74 deletions(-) create mode 100644 client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java 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 9fa83e458..2b16a92be 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -1,12 +1,16 @@ package com.clickhouse.data; +import java.lang.reflect.Array; import java.math.BigDecimal; import java.math.BigInteger; import java.net.Inet4Address; import java.net.Inet6Address; import java.time.LocalDate; import java.time.LocalDateTime; +import java.time.ZonedDateTime; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.EnumSet; @@ -20,7 +24,12 @@ import java.util.TreeMap; import java.util.TreeSet; 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; @@ -158,24 +167,54 @@ public static Map, Integer> buildVariantMapping(List>> dataTypeClassMap() { Map>> map = new HashMap<>(); - map.put(UInt256, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class, BigInteger.class)))); - map.put(Int256, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class, BigInteger.class)))); - map.put(UInt128, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class, BigInteger.class)))); - map.put(Int128, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class, BigInteger.class)))); - map.put(UInt64, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class)))); - map.put(Int64, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class)))); - map.put(UInt32, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class )))); - map.put(Int32, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class)))); - map.put(UInt16, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class)))); - map.put(Int16, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class, short.class, Short.class)))); - map.put(UInt8, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class)))); - map.put(Int8, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class)))); - - map.put(String, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(String.class)))); - + // We allow to write short to UInt8 even it may not fit. It is done because we have to allow users to utilize UInt* data types. + List> allNumberClassesOrderedBySize = Arrays.asList(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class, BigInteger.class); + Set> setOfAllNumberClasses = Collections.unmodifiableSet(new HashSet<>(allNumberClassesOrderedBySize)); + map.put(UInt256, setOfAllNumberClasses); + map.put(Int256, setOfAllNumberClasses); + map.put(UInt128, setOfAllNumberClasses); + map.put(Int128, setOfAllNumberClasses); + map.put(UInt64, setOfAllNumberClasses); + + map.put(Int64, setOf(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class)); + map.put(UInt32, setOf(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class )); + map.put(Int32, setOf(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class)); + map.put(UInt16, setOf(byte.class, Byte.class, short.class, Short.class, int.class, Integer.class)); + map.put(Int16, setOf(byte.class, Byte.class, short.class, Short.class)); + map.put(UInt8, setOf(byte.class, Byte.class, short.class, Short.class)); + map.put(Int8, setOf(byte.class, Byte.class)); + + map.put(Bool, setOf(boolean.class, Boolean.class)); + map.put(String, setOf(String.class)); + map.put(Float64, setOf(float.class, Float.class, double.class, Double.class)); + map.put(Float32, setOf(float.class, Float.class)); + map.put(Decimal, setOf(float.class, Float.class, double.class, Double.class, BigDecimal.class)); + map.put(Decimal256, setOf(float.class, Float.class, double.class, Double.class, BigDecimal.class)); + map.put(Decimal128, setOf(float.class, Float.class, double.class, Double.class, BigDecimal.class)); + map.put(Decimal64, setOf(float.class, Float.class, double.class, Double.class)); + map.put(Decimal32, setOf(float.class, Float.class)); + + map.put(IPv4, setOf(Inet4Address.class)); + map.put(IPv6, setOf(Inet6Address.class)); + map.put(UUID, setOf(java.util.UUID.class)); + + map.put(Point, setOf(double[].class, ClickHouseGeoPointValue.class)); + map.put(Ring, setOf(double[][].class, ClickHouseGeoRingValue.class)); + map.put(Polygon, setOf(double[][][].class, ClickHouseGeoPolygonValue.class)); + map.put(MultiPolygon, setOf(double[][][][].class, ClickHouseGeoMultiPolygonValue.class)); + + map.put(Date, setOf(LocalDateTime.class, LocalDate.class, ZonedDateTime.class)); + map.put(Date32, setOf(LocalDateTime.class, LocalDate.class, ZonedDateTime.class)); + map.put(DateTime64, setOf(LocalDateTime.class, ZonedDateTime.class)); + map.put(DateTime32, setOf(LocalDateTime.class, ZonedDateTime.class)); + map.put(DateTime, setOf(LocalDateTime.class, ZonedDateTime.class)); return map; } + private static Set> setOf(Class... args) { + return Collections.unmodifiableSet(new HashSet<>(Arrays.stream(args).collect(Collectors.toList()))); + } + /** * Immutable set(sorted) for all aliases. 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 4ce4a33de..53c871550 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 @@ -8,6 +8,10 @@ import com.clickhouse.data.ClickHouseDataType; import com.clickhouse.data.format.BinaryStreamUtils; import com.clickhouse.data.value.ClickHouseBitmap; +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 org.objectweb.asm.ClassWriter; import org.objectweb.asm.MethodVisitor; import org.objectweb.asm.Opcodes; @@ -17,6 +21,7 @@ import java.io.IOException; import java.io.OutputStream; +import java.lang.reflect.Array; import java.lang.reflect.Method; import java.math.BigDecimal; import java.math.BigInteger; @@ -28,17 +33,13 @@ import java.time.ZoneId; import java.time.ZonedDateTime; import java.util.Arrays; -import java.util.Collection; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.StringTokenizer; -import java.util.TimeZone; import java.util.UUID; -import java.util.function.Function; import java.util.stream.Collectors; import static org.objectweb.asm.Opcodes.ACC_PUBLIC; @@ -60,7 +61,7 @@ public static void serializeData(OutputStream stream, Object value, ClickHouseCo serializeArrayData(stream, value, column); break; case Tuple: - serializeTupleData(stream, value, column); + serializeTuple(stream, value, column); break; case Map: serializeMapData(stream, value, column); @@ -68,6 +69,25 @@ public static void serializeData(OutputStream stream, Object value, ClickHouseCo case AggregateFunction: serializeAggregateFunction(stream, value, column); break; + case Variant: + serializerVariant(stream, column, value); + break; + case Point: + value = value instanceof ClickHouseGeoPointValue ? ((ClickHouseGeoPointValue)value).getValue() : value; + serializeTuple(stream, value, GEO_POINT_TUPLE); + break; + case Ring: + value = value instanceof ClickHouseGeoRingValue ? ((ClickHouseGeoRingValue)value).getValue() : value; + serializeArrayData(stream, value, GEO_RING_ARRAY); + break; + case Polygon: + value = value instanceof ClickHouseGeoPolygonValue ? ((ClickHouseGeoPolygonValue)value).getValue() : value; + serializeArrayData(stream, value, GEO_POLYGON_ARRAY); + break; + case MultiPolygon: + value = value instanceof ClickHouseGeoMultiPolygonValue ? ((ClickHouseGeoMultiPolygonValue)value).getValue() : value; + serializeArrayData(stream, value, GEO_MULTI_POLYGON_ARRAY); + break; default: serializePrimitiveData(stream, value, column); break; @@ -76,23 +96,39 @@ public static void serializeData(OutputStream stream, Object value, ClickHouseCo } private static void serializeArrayData(OutputStream stream, Object value, ClickHouseColumn column) throws IOException { - //Serialize the array to the stream - //The array is a list of values - List values = (List) value; - writeVarInt(stream, values.size()); - for (Object val : values) { - if (column.getArrayBaseColumn().isNullable()) { - if (val == null) { - writeNull(stream); - continue; + + if (value instanceof List) { + //Serialize the array to the stream + //The array is a list of values + List values = (List) value; + writeVarInt(stream, values.size()); + for (Object val : values) { + if (column.getArrayBaseColumn().isNullable()) { + if (val == null) { + writeNull(stream); + continue; + } + writeNonNull(stream); + } + serializeData(stream, val, column.getArrayBaseColumn()); + } + } else if (value.getClass().isArray()) { + writeVarInt(stream, Array.getLength(value)); + for (int i = 0; i < Array.getLength(value); i++) { + Object val = Array.get(value, i); + if (column.getArrayBaseColumn().isNullable()) { + if (val == null) { + writeNull(stream); + continue; + } + writeNonNull(stream); } - writeNonNull(stream); + serializeData(stream, val, column.getNestedColumns().get(0)); } - serializeData(stream, val, column.getArrayBaseColumn()); } } - private static void serializeTupleData(OutputStream stream, Object value, ClickHouseColumn column) throws IOException { + private static void serializeTuple(OutputStream stream, Object value, ClickHouseColumn column) throws IOException { //Serialize the tuple to the stream //The tuple is a list of values if (value instanceof List) { @@ -100,10 +136,16 @@ private static void serializeTupleData(OutputStream stream, Object value, ClickH for (int i = 0; i < values.size(); i++) { serializeData(stream, values.get(i), column.getNestedColumns().get(i)); } - } else if (value instanceof Object[]) { - Object[] values = (Object[]) value; - for (int i = 0; i < values.length; i++) { - serializeData(stream, values[i], column.getNestedColumns().get(i)); + } +// else if (value instanceof Object[]) { +// Object[] values = (Object[]) value; +// for (int i = 0; i < values.length; i++) { +// serializeData(stream, values[i], column.getNestedColumns().get(i)); +// } +// } + else if (value.getClass().isArray()) { + for (int i = 0; i < Array.getLength(value); i++) { + serializeData(stream, Array.get(value, i), column.getNestedColumns().get(i)); } } else { throw new IllegalArgumentException("Cannot serialize " + value + " as a tuple"); @@ -165,10 +207,10 @@ private static void serializePrimitiveData(OutputStream stream, Object value, Cl BinaryStreamUtils.writeUnsignedInt256(stream, convertToBigInteger(value)); break; case Float32: - BinaryStreamUtils.writeFloat32(stream, (Float) value); + BinaryStreamUtils.writeFloat32(stream, (float) value); break; case Float64: - BinaryStreamUtils.writeFloat64(stream, (Double) value); + BinaryStreamUtils.writeFloat64(stream, (double) value); break; case Decimal: case Decimal32: @@ -220,15 +262,6 @@ private static void serializePrimitiveData(OutputStream stream, Object value, Cl case JSON: serializeJSON(stream, value); break; - case Tuple: - serializeTuple(stream, column, (Object[]) value); - break; - case Point: - serializeTuple(stream, GEO_POINT_TUPLE, (Object[]) value); - break; - case Variant: - serializerVariant(stream, column, value); - break; default: throw new UnsupportedOperationException("Unsupported data type: " + column.getDataType()); } @@ -241,18 +274,18 @@ private static void serializeJSON(OutputStream stream, Object value) throws IOEx throw new UnsupportedOperationException("Serialization of Java object to JSON is not supported yet."); } } - - private static void serializeTuple(OutputStream out, ClickHouseColumn column, Object[] tupleValues) throws IOException { - if (column.getNestedColumns().size() != tupleValues.length) { - throw new IllegalArgumentException("Column " + column.getColumnName() + " defines as Tuple with " - + column.getNestedColumns().size() +" elements, but only " + tupleValues.length + " provided"); - } - - List nested = column.getNestedColumns(); - for (int i = 0; i < nested.size() ; i++) { - serializeData(out, tupleValues[i], nested.get(i)); - } - } +// +// private static void serializeTuple(OutputStream out, ClickHouseColumn column, Object[] tupleValues) throws IOException { +// if (column.getNestedColumns().size() != tupleValues.length) { +// throw new IllegalArgumentException("Column " + column.getColumnName() + " defines as Tuple with " +// + column.getNestedColumns().size() +" elements, but only " + tupleValues.length + " provided"); +// } +// +// List nested = column.getNestedColumns(); +// for (int i = 0; i < nested.size() ; i++) { +// serializeData(out, tupleValues[i], nested.get(i)); +// } +// } private static void serializerVariant(OutputStream out, ClickHouseColumn column, Object value) throws IOException { @@ -266,6 +299,9 @@ private static void serializerVariant(OutputStream out, ClickHouseColumn column, } private static final ClickHouseColumn GEO_POINT_TUPLE = ClickHouseColumn.parse("geopoint Tuple(Float64, Float64)").get(0); + private static final ClickHouseColumn GEO_RING_ARRAY = ClickHouseColumn.parse("georing Array(Tuple(Float64, Float64))").get(0); + private static final ClickHouseColumn GEO_POLYGON_ARRAY = ClickHouseColumn.parse("geopolygin Array(Array(Tuple(Float64, Float64)))").get(0); + private static final ClickHouseColumn GEO_MULTI_POLYGON_ARRAY = ClickHouseColumn.parse("geomultipolygin Array(Array(Array(Tuple(Float64, Float64))))").get(0); private static void serializeAggregateFunction(OutputStream stream, Object value, ClickHouseColumn column) throws IOException { if (column.getAggregateFunction() == ClickHouseAggregateFunction.groupBitmap) { 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 57d70ec3b..f98029156 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,16 +8,21 @@ import com.clickhouse.client.api.enums.Protocol; import com.clickhouse.client.api.insert.InsertSettings; import com.clickhouse.client.api.query.GenericRecord; -import com.clickhouse.client.api.query.QueryResponse; +import com.clickhouse.data.ClickHouseDataType; +import lombok.AllArgsConstructor; +import lombok.Data; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; import java.io.IOException; +import java.lang.reflect.Method; +import java.time.LocalDateTime; +import java.time.temporal.ChronoUnit; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Map; public class DataTypeTests extends BaseIntegrationTest { @@ -81,24 +86,137 @@ public void testNestedDataTypes() throws Exception { } @Test - public void testVariantDataTypeWithPrimitives() throws Exception { + public void testVariantWithSimpleDataTypes() throws Exception { final String table = "test_variant_primitives"; - String tblCreateSQL = VariantDTO.tblCreateSQL(table); - client.execute("DROP TABLE IF EXISTS " + table).get(); - client.execute(tblCreateSQL, (CommandSettings) new CommandSettings().serverSetting("enable_variant_type", "1")); + final DataTypesTestingPOJO sample = new DataTypesTestingPOJO(); + System.out.println("sample: " + sample); + + dataTypesLoop: + for (ClickHouseDataType dataType : ClickHouseDataType.values()) { + System.out.println("Testing " + dataType); + client.execute("DROP TABLE IF EXISTS " + table).get(); + StringBuilder b = new StringBuilder(" CREATE TABLE "); + b.append(table).append(" ( rowId Int64, field Variant(String, ").append(dataType.name()); + + switch (dataType) { + case String: + case FixedString: + case IntervalYear: + case IntervalDay: + case IntervalHour: + case IntervalWeek: + case IntervalMonth: + case IntervalMinute: + case IntervalSecond: + case IntervalNanosecond: + case IntervalMicrosecond: + case IntervalQuarter: + case IntervalMillisecond: + case Nothing: + case Variant: + case JSON: + case Object: + // skipped + continue dataTypesLoop; + + case Decimal: + case Decimal32: + case Decimal64: + case Decimal128: + case Decimal256: + case Array: + case Map: + case Nested: + case Tuple: + case SimpleAggregateFunction: + case AggregateFunction: + case Enum8: + case Enum16: + // tested separately + continue dataTypesLoop; + + } + b.append(")) Engine = MergeTree ORDER BY ()"); + + client.execute(b.toString(), (CommandSettings) new CommandSettings().serverSetting("enable_variant_type", "1")); + client.register(DTOForVariantPrimitivesTests.class, client.getTableSchema(table)); + + Object value = null; + for (Method m : sample.getClass().getDeclaredMethods()) { + if (m.getName().equalsIgnoreCase("get" + dataType.name())) { + value = m.invoke(sample); + System.out.println("selected " + value + " returned by method " + m.getName()); + break; + } + } + + List data = new ArrayList<>(); + data.add(new DTOForVariantPrimitivesTests(0, value)); + client.insert(table, data).get().close(); + + List rows = client.queryAll("SELECT * FROM " + table); + for (GenericRecord row : rows) { + String strValue = row.getString("field"); + switch (dataType) { + case Date: + case Date32: + strValue = row.getLocalDate("field").toString(); + break; + case DateTime64: + case DateTime: + case DateTime32: + strValue = row.getLocalDateTime("field").truncatedTo(ChronoUnit.SECONDS).toString(); + value = ((LocalDateTime)value ).truncatedTo(ChronoUnit.SECONDS).toString(); + break; + case Point: + strValue = row.getGeoPoint("field").toString(); + break; + case Ring: + strValue = row.getGeoRing("field").toString(); + break; + case Polygon: + strValue = row.getGeoPolygon("field").toString(); + break; + case MultiPolygon: + strValue = row.getGeoMultiPolygon("field").toString(); + break; + } + System.out.println("field: " + strValue + " value " + value); + if (value.getClass().isPrimitive()) { + Assert.assertEquals(strValue, String.valueOf(value)); + } else { + Assert.assertEquals(strValue, String.valueOf(value)); + } + } + } + } - client.register(VariantDTO.class, client.getTableSchema(table)); + @Data + @AllArgsConstructor + public static class DTOForVariantPrimitivesTests { + private int rowId; + private Object field; + } -// List data = Arrays.asList(new VariantDTO(1, (short)200), new VariantDTO(2, (byte)127), new VariantDTO(3, "test ☺")); - List data = Arrays.asList(new VariantDTO(1, (short)200), new VariantDTO(2, (byte)127)); - client.insert(table, data); + public void testVariantWithDecimals() throws Exception { - List rows = client.queryAll("SELECT * FROM " + table); - for (GenericRecord row : rows) { - System.out.println(row.getInteger("rowId") + " " + row.getInteger("a")); - } } + public void testVariantWithDateTime() throws Exception { + + } + + public void testVariantWithNullable() throws Exception { + + } + + public void testVariantWithArrays() throws Exception { + + } + + public void testVariantWithMaps() throws Exception { + + } public static String tableDefinition(String table, String... columns) { StringBuilder sb = new StringBuilder(); diff --git a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java new file mode 100644 index 000000000..90b6594ce --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java @@ -0,0 +1,283 @@ +package com.clickhouse.client.datatypes; + +import com.clickhouse.data.value.ClickHouseBitmap; +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 lombok.Data; +import lombok.ToString; +import org.apache.commons.lang3.RandomStringUtils; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.Inet4Address; +import java.net.Inet6Address; +import java.net.UnknownHostException; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.UUID; + + +@Data +@ToString +public class DataTypesTestingPOJO { + private byte byteValue; + private Byte boxedByte; + private byte int8; + private short int16; + private Short boxedShort; + private int int32; + private Integer boxedInt; + private long int64; + private Long boxedLong; + private long int64_default; + private BigInteger int128; + private BigInteger int256; + + private short uInt8; + private int uInt16; + private long uInt32; + private BigInteger uInt64; + private BigInteger uInt128; + private BigInteger uInt256; + + private float float32; + private Float boxedFloat; + private double float64; + private Double boxedDouble; + + private BigDecimal decimal32; + private BigDecimal decimal64; + private BigDecimal decimal128; + private BigDecimal decimal256; + + private boolean bool; + private Boolean boxedBool; + + private String string; + private String fixedString; + + private LocalDate date; + private LocalDate date32; + + private LocalDateTime dateTime; + private LocalDateTime dateTime32; + private LocalDateTime dateTime64; + + private UUID UUID; + + private byte enum8; + private int enum16; + + private Inet4Address IPv4; + private Inet6Address IPv6; + + private ClickHouseGeoPointValue point; + private ClickHouseGeoPolygonValue polygon; + private ClickHouseGeoRingValue ring; + private ClickHouseGeoMultiPolygonValue multiPolygon; + + private List array; + private List tuple; + private Map map; + private List nestedInnerInt; + private List nestedInnerString; + private List nestedInnerNullableInt; + + private ClickHouseBitmap groupBitmapUint32; + private ClickHouseBitmap groupBitmapUint64; + + public DataTypesTestingPOJO() { + final Random random = new Random(); + byteValue = (byte) random.nextInt(); + boxedByte = (byte) random.nextInt(); + int8 = (byte) random.nextInt(128); + int16 = (short)random.nextInt(32768); + boxedShort = (short) random.nextInt(); + int32 = random.nextInt(); + boxedInt = random.nextInt(); + int64 = random.nextLong(); + boxedLong = random.nextLong(); + BigInteger upper = BigInteger.valueOf(random.nextLong()).shiftLeft(64); + BigInteger lower = BigInteger.valueOf(random.nextLong()).and(BigInteger.valueOf(Long.MAX_VALUE)); + + int128 = upper.or(lower); + + BigInteger upper1 = BigInteger.valueOf(random.nextLong()).shiftLeft(192); + BigInteger upper2 = BigInteger.valueOf(random.nextLong()).shiftLeft(128); + BigInteger lower1 = BigInteger.valueOf(random.nextLong()).shiftLeft(64); + BigInteger lower2 = BigInteger.valueOf(random.nextLong()).and(BigInteger.valueOf(Long.MAX_VALUE)); + + int256 = upper1.or(upper2).or(lower1).or(lower2); + + uInt8 = (short) random.nextInt(255); + uInt16 = random.nextInt(32768); + uInt32 = (long) (random.nextDouble() * 4294967295L); + uInt64 = BigInteger.valueOf((long) (random.nextDouble() * 18446744073709615L)); + + + uInt128 = upper.or(lower).abs(); + uInt256 = upper1.or(upper2).or(lower1).or(lower2).abs(); + + float32 = random.nextFloat(); + float64 = random.nextDouble(); + boxedFloat = random.nextFloat(); + boxedDouble = random.nextDouble(); + + decimal32 = BigDecimal.valueOf(random.nextDouble()); + decimal64 = BigDecimal.valueOf(random.nextDouble()); + decimal128 = BigDecimal.valueOf(random.nextDouble()); + decimal256 = BigDecimal.valueOf(random.nextDouble()); + + bool = random.nextBoolean(); + + string = RandomStringUtils.randomAlphabetic(1, 256); + fixedString = RandomStringUtils.randomAlphabetic(3); + + date = LocalDate.now(); + date32 = LocalDate.now(); + + dateTime = LocalDateTime.now(); + dateTime32 = LocalDateTime.now(); + dateTime64 = LocalDateTime.now(); + + UUID = UUID.randomUUID(); + + enum8 = (byte) random.nextInt(27); + enum16 = random.nextInt(27); + + try { + byte[] addr4 = new byte[4]; + random.nextBytes(addr4); + IPv4 = (Inet4Address) Inet4Address.getByAddress(addr4); + + byte[] addr6 = new byte[16]; + random.nextBytes(addr6); + IPv6 = (Inet6Address) Inet6Address.getByAddress(addr6); + } catch (UnknownHostException e) { + IPv4 = null; + IPv6 = null; + } + + point = ClickHouseGeoPointValue.of(new double[]{random.nextFloat(), random.nextFloat()}); + polygon = ClickHouseGeoPolygonValue.of(new double[][][]{ + { + {random.nextFloat(), random.nextFloat()}, + {random.nextFloat(), random.nextFloat()} + }, + }); + ring = ClickHouseGeoRingValue.of(new double[][]{ + {random.nextFloat(), random.nextFloat()}, + {random.nextFloat(), random.nextFloat()}, + {random.nextFloat(), random.nextFloat()}}); + multiPolygon = ClickHouseGeoMultiPolygonValue.of(new double[][][][]{ + { + { + {random.nextFloat(), random.nextFloat()}, + {random.nextFloat(), random.nextFloat()} + }, + }, + { + { + {random.nextFloat(), random.nextFloat()}, + {random.nextFloat(), random.nextFloat()} + }, + } + + } + ); + + array = Arrays.asList("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o", "p", "q", "r", "s", "t", "u", "v", "w", "x", "y", "z"); + tuple = Arrays.asList(uInt64, int32, string); + map = new HashMap<>(); + for (int i = 0; i < 10; i++) { + map.put(String.valueOf((char) ('a' + i)), i + 1); + } + + List innerInt = new ArrayList<>(); + innerInt.add(random.nextInt(Integer.MAX_VALUE)); + innerInt.add(random.nextInt(Integer.MAX_VALUE)); + nestedInnerInt = innerInt; + + List innerString = new ArrayList<>(); + innerString.add(RandomStringUtils.randomAlphabetic(1, 256)); + innerString.add(RandomStringUtils.randomAlphabetic(1, 256)); + nestedInnerString = innerString; + + List innerNullableInt = new ArrayList<>(); + innerNullableInt.add(null); + innerNullableInt.add(random.nextInt(Integer.MAX_VALUE)); + nestedInnerNullableInt = innerNullableInt; + + groupBitmapUint32 = ClickHouseBitmap.wrap(random.ints(5, Integer.MAX_VALUE - 100, Integer.MAX_VALUE).toArray()); + groupBitmapUint64 = ClickHouseBitmap.wrap(random.longs(5, Long.MAX_VALUE - 100, Long.MAX_VALUE).toArray()); + + } + + public boolean getBool() { + return bool; + } + + public static String generateTableCreateSQL(String tableName) { + return "CREATE TABLE " + tableName + " (" + + "byteValue Int8," + + "int8 Int8, " + + "boxedByte Int8, " + + "int8_default Int8 DEFAULT 0, " + + "int16 Int16, " + + "boxedShort Int16, " + + "int16_default Int16 DEFAULT 0, " + + "int32 Int32, " + + "boxedInt Int32, " + + "int32_default Int32 DEFAULT 0, " + + "int64 Int64, " + + "boxedLong Int64, " + + "int64_default Int64 DEFAULT 0, " + + "int128 Int128, " + + "int128_default Int128 DEFAULT 0, " + + "int256 Int256, " + + "int256_default Int256 DEFAULT 0, " + + "uint8 UInt8, " + + "uint16 UInt16, " + + "uint32 UInt32, " + + "uint64 UInt64, " + + "uint128 UInt128, " + + "uint256 UInt256, " + + "float32 Float32, " + + "boxedFloat Float32, " + + "float64 Float64, " + + "boxedDouble Float64, " + + "decimal32 Decimal32(2), " + + "decimal64 Decimal64(3), " + + "decimal128 Decimal128(4), " + + "decimal256 Decimal256(5), " + + "bool UInt8, " + +// "boxedBool UInt8, " + + "string String, " + + "fixedString FixedString(3), " + + "date Date, " + + "date32 Date, " + + "dateTime DateTime, " + + "dateTime64 DateTime64(3), " + + "uuid UUID, " + + "enum8 Enum8('a' = 1, 'b' = 2, 'c' = 3, 'd' = 4, 'e' = 5, 'f' = 6, 'g' = 7, 'h' = 8, 'i' = 9, 'j' = 10, 'k' = 11, 'l' = 12, 'm' = 13, 'n' = 14, 'o' = 15, 'p' = 16, 'q' = 17, 'r' = 18, 's' = 19, 't' = 20, 'u' = 21, 'v' = 22, 'w' = 23, 'x' = 24, 'y' = 25, 'z' = 26), " + + "enum16 Enum16('a' = 1, 'b' = 2, 'c' = 3, 'd' = 4, 'e' = 5, 'f' = 6, 'g' = 7, 'h' = 8, 'i' = 9, 'j' = 10, 'k' = 11, 'l' = 12, 'm' = 13, 'n' = 14, 'o' = 15, 'p' = 16, 'q' = 17, 'r' = 18, 's' = 19, 't' = 20, 'u' = 21, 'v' = 22, 'w' = 23, 'x' = 24, 'y' = 25, 'z' = 26), " + + "ipv4 IPv4, " + + "ipv6 IPv6, " + + "array Array(String), " + + "tuple Tuple(UInt64, Int32, String), " + + "map Map(String, Int32), " + + "nested Nested (innerInt Int32, innerString String, " + + "innerNullableInt Nullable(Int32)), " + + "groupBitmapUint32 AggregateFunction(groupBitmap, UInt32), " + + "groupBitmapUint64 AggregateFunction(groupBitmap, UInt64) " + + ") ENGINE = MergeTree ORDER BY ()"; + } +} diff --git a/client-v2/src/test/java/com/clickhouse/client/datatypes/VariantDTO.java b/client-v2/src/test/java/com/clickhouse/client/datatypes/VariantDTO.java index e48be26a6..cd36f76bc 100644 --- a/client-v2/src/test/java/com/clickhouse/client/datatypes/VariantDTO.java +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/VariantDTO.java @@ -16,7 +16,24 @@ public class VariantDTO { private Object a; + private Object b; + + private Object c; + + private Object d; + + private Object e; + + private Object f; + public static String tblCreateSQL(String table) { - return tableDefinition(table, "rowId Int16, a Variant(String, Int16)"); + return tableDefinition(table, + "rowId Int16", + "a Variant(String, Int16)", + "b Variant(String, Int128)", + "c Variant(String, Decimal128(4))", + "d Variant(String, Float32)", + "e Variant(Int128, Decimal128(4))", + "f Variant(Float64, Int128)"); } } diff --git a/client-v2/src/test/java/com/clickhouse/client/insert/SamplePOJO.java b/client-v2/src/test/java/com/clickhouse/client/insert/SamplePOJO.java index 47d260256..7d60c82df 100644 --- a/client-v2/src/test/java/com/clickhouse/client/insert/SamplePOJO.java +++ b/client-v2/src/test/java/com/clickhouse/client/insert/SamplePOJO.java @@ -242,6 +242,10 @@ public Boolean getBoxedBool() { return boxedBool; } + public boolean getBool() { + return true; + } + public void setBoxedBool(Boolean boxedBool) { this.boxedBool = boxedBool; } From 004d604341e3defc22e5e76e773163561d1f9171 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 23 Jan 2025 14:07:17 -0800 Subject: [PATCH 4/7] implemented enum support for variant. some fixes --- .../clickhouse/data/ClickHouseDataType.java | 3 + .../internal/AbstractBinaryFormatReader.java | 28 ++++-- .../internal/MapBackedRecord.java | 17 +--- .../internal/SerializerUtils.java | 62 ++++++------ .../client/datatypes/DataTypeTests.java | 97 +++++++++++++++++-- .../clickhouse/client/query/QueryTests.java | 6 +- 6 files changed, 154 insertions(+), 59 deletions(-) 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 2b16a92be..d2515ea3f 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -208,6 +208,9 @@ static Map>> dataTypeClassMap() { map.put(DateTime64, setOf(LocalDateTime.class, ZonedDateTime.class)); map.put(DateTime32, setOf(LocalDateTime.class, ZonedDateTime.class)); map.put(DateTime, setOf(LocalDateTime.class, ZonedDateTime.class)); + + map.put(Enum8, setOf(java.lang.String.class,byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class)); + map.put(Enum16, setOf(java.lang.String.class,byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class)); return map; } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java index 3bca484fc..b1e6be534 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java @@ -271,23 +271,37 @@ public TableSchema getSchema() { @Override public String getString(String colName) { - Object value = readValue(colName); + return readAsString(readValue(colName), schema.getColumnByName(colName)); + } + + /** + * Converts value in to a string representation. Does some formatting for selected data types + * @return string representation of a value for specified column + */ + public static String readAsString(Object value, ClickHouseColumn column) { if (value == null) { return null; } else if (value instanceof String) { return (String) value; } else if (value instanceof ZonedDateTime) { - ClickHouseDataType dataType = schema.getColumnByName(colName).getDataType(); + ClickHouseDataType dataType = column.getDataType(); ZonedDateTime zdt = (ZonedDateTime) value; if (dataType == ClickHouseDataType.Date) { return zdt.format(com.clickhouse.client.api.DataTypeUtils.DATE_FORMATTER).toString(); } return value.toString(); - } else { - ClickHouseDataType dataType = schema.getColumnByName(colName).getDataType(); - if (dataType == ClickHouseDataType.Enum8 || dataType == ClickHouseDataType.Enum16) { - ClickHouseEnum clickHouseEnum = schema.getColumnByName(colName).getEnumConstants(); - return clickHouseEnum.name(Integer.parseInt(value.toString())); + } else if (value instanceof Number ) { + ClickHouseDataType dataType = column.getDataType(); + int num = ((Number)value).intValue(); + if (column.getDataType() == ClickHouseDataType.Variant) { + for (ClickHouseColumn c : column.getNestedColumns()) { + // TODO: will work only if single enum listed as variant + if (c.getDataType() == ClickHouseDataType.Enum8 || c.getDataType() == ClickHouseDataType.Enum16) { + return c.getEnumConstants().name(num); + } + } + } else if (dataType == ClickHouseDataType.Enum8 || dataType == ClickHouseDataType.Enum16) { + return column.getEnumConstants().name(num); } } return value.toString(); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java index 6c514c4a6..aee8ff48d 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java @@ -47,25 +47,12 @@ public T readValue(String colName) { @Override public String getString(String colName) { - Object value = readValue(colName); - if (value == null) { - return null; - } else if (value instanceof String) { - return (String) value; - } - return value.toString(); + return AbstractBinaryFormatReader.readAsString(readValue(colName), schema.getColumnByName(colName)); } @Override public String getString(int index) { - // TODO: it may be incorrect to call .toString() on some objects - Object value = readValue(index); - if (value == null) { - return null; - } else if (value instanceof String) { - return (String) value; - } - return value.toString(); + return getString(schema.columnIndexToName(index)); } private T readNumberValue(String colName, NumberConverter.NumberType targetType) { 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 53c871550..06bc84abf 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 @@ -2,6 +2,8 @@ import com.clickhouse.client.api.Client; import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.data_formats.RowBinaryFormatSerializer; +import com.clickhouse.client.api.data_formats.RowBinaryFormatWriter; import com.clickhouse.client.api.query.POJOSetter; import com.clickhouse.data.ClickHouseAggregateFunction; import com.clickhouse.data.ClickHouseColumn; @@ -61,7 +63,7 @@ public static void serializeData(OutputStream stream, Object value, ClickHouseCo serializeArrayData(stream, value, column); break; case Tuple: - serializeTuple(stream, value, column); + serializeTupleData(stream, value, column); break; case Map: serializeMapData(stream, value, column); @@ -74,7 +76,7 @@ public static void serializeData(OutputStream stream, Object value, ClickHouseCo break; case Point: value = value instanceof ClickHouseGeoPointValue ? ((ClickHouseGeoPointValue)value).getValue() : value; - serializeTuple(stream, value, GEO_POINT_TUPLE); + serializeTupleData(stream, value, GEO_POINT_TUPLE); break; case Ring: value = value instanceof ClickHouseGeoRingValue ? ((ClickHouseGeoRingValue)value).getValue() : value; @@ -128,7 +130,7 @@ private static void serializeArrayData(OutputStream stream, Object value, ClickH } } - private static void serializeTuple(OutputStream stream, Object value, ClickHouseColumn column) throws IOException { + private static void serializeTupleData(OutputStream stream, Object value, ClickHouseColumn column) throws IOException { //Serialize the tuple to the stream //The tuple is a list of values if (value instanceof List) { @@ -136,14 +138,8 @@ private static void serializeTuple(OutputStream stream, Object value, ClickHouse for (int i = 0; i < values.size(); i++) { serializeData(stream, values.get(i), column.getNestedColumns().get(i)); } - } -// else if (value instanceof Object[]) { -// Object[] values = (Object[]) value; -// for (int i = 0; i < values.length; i++) { -// serializeData(stream, values[i], column.getNestedColumns().get(i)); -// } -// } - else if (value.getClass().isArray()) { + } else if (value.getClass().isArray()) { + // TODO: this code uses reflection - we might need to measure it and find faster solution. for (int i = 0; i < Array.getLength(value); i++) { serializeData(stream, Array.get(value, i), column.getNestedColumns().get(i)); } @@ -217,7 +213,7 @@ private static void serializePrimitiveData(OutputStream stream, Object value, Cl case Decimal64: case Decimal128: case Decimal256: - BinaryStreamUtils.writeDecimal(stream, (BigDecimal) value, column.getPrecision(), column.getScale()); + BinaryStreamUtils.writeDecimal(stream, convertToBigDecimal(value), column.getPrecision(), column.getScale()); break; case Bool: BinaryStreamUtils.writeBoolean(stream, (Boolean) value); @@ -247,11 +243,15 @@ private static void serializePrimitiveData(OutputStream stream, Object value, Cl case UUID: BinaryStreamUtils.writeUuid(stream, (UUID) value); break; +// case Enum8: +// BinaryStreamUtils.writeEnum8(stream, (Byte) value); +// break; +// case Enum16: +// BinaryStreamUtils.writeEnum16(stream, convertToInteger(value)); +// break; case Enum8: - BinaryStreamUtils.writeEnum8(stream, (Byte) value); - break; case Enum16: - BinaryStreamUtils.writeEnum16(stream, convertToInteger(value)); + serializeEnumData(stream, column, value); break; case IPv4: BinaryStreamUtils.writeInet4Address(stream, (Inet4Address) value); @@ -267,6 +267,25 @@ private static void serializePrimitiveData(OutputStream stream, Object value, Cl } } + private static void serializeEnumData(OutputStream stream, ClickHouseColumn column, Object value) throws IOException { + int enumValue = -1; + if (value instanceof String) { + enumValue = column.getEnumConstants().value((String) value); + } else if (value instanceof Number) { + enumValue = ((Number)value).intValue(); + } else { + throw new IllegalArgumentException("Cannot write value of class " + value.getClass() + " into column with Enum type " + column.getOriginalTypeName()); + } + + if (column.getDataType() == ClickHouseDataType.Enum8) { + BinaryStreamUtils.writeInt8(stream, enumValue); + } else if (column.getDataType() == ClickHouseDataType.Enum16) { + BinaryStreamUtils.writeInt16(stream, enumValue); + } else { + throw new ClientException("Bug! serializeEnumData() was called for " + column.getDataType()); + } + } + private static void serializeJSON(OutputStream stream, Object value) throws IOException { if (value instanceof String) { BinaryStreamUtils.writeString(stream, (String)value); @@ -274,19 +293,6 @@ private static void serializeJSON(OutputStream stream, Object value) throws IOEx throw new UnsupportedOperationException("Serialization of Java object to JSON is not supported yet."); } } -// -// private static void serializeTuple(OutputStream out, ClickHouseColumn column, Object[] tupleValues) throws IOException { -// if (column.getNestedColumns().size() != tupleValues.length) { -// throw new IllegalArgumentException("Column " + column.getColumnName() + " defines as Tuple with " -// + column.getNestedColumns().size() +" elements, but only " + tupleValues.length + " provided"); -// } -// -// List nested = column.getNestedColumns(); -// for (int i = 0; i < nested.size() ; i++) { -// serializeData(out, tupleValues[i], nested.get(i)); -// } -// } - private static void serializerVariant(OutputStream out, ClickHouseColumn column, Object value) throws IOException { int typeOrdNum = column.getVariantOrdNum(value); 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 f98029156..3d1600748 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 @@ -6,6 +6,7 @@ import com.clickhouse.client.api.Client; import com.clickhouse.client.api.command.CommandSettings; import com.clickhouse.client.api.enums.Protocol; +import com.clickhouse.client.api.insert.InsertResponse; import com.clickhouse.client.api.insert.InsertSettings; import com.clickhouse.client.api.query.GenericRecord; import com.clickhouse.data.ClickHouseDataType; @@ -22,6 +23,7 @@ import java.time.temporal.ChronoUnit; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.List; public class DataTypeTests extends BaseIntegrationTest { @@ -63,7 +65,7 @@ public void tearDown() { } - @Test + @Test(groups = {"integration"}) public void testNestedDataTypes() throws Exception { final String table = "test_nested_types"; String tblCreateSQL = NestedTypesDTO.tblCreateSQL(table); @@ -85,7 +87,7 @@ public void testNestedDataTypes() throws Exception { } - @Test + @Test(groups = {"integration"}) public void testVariantWithSimpleDataTypes() throws Exception { final String table = "test_variant_primitives"; final DataTypesTestingPOJO sample = new DataTypesTestingPOJO(); @@ -198,24 +200,103 @@ public static class DTOForVariantPrimitivesTests { private Object field; } + @Test(groups = {"integration"}) public void testVariantWithDecimals() throws Exception { - + testVariantWith("decimals", new String[]{"field Variant(String, Decimal(4, 4))"}, + new Object[]{ + "10.2", + 10.2d, // TODO: when f it gives 10.199 + }, + new String[]{ + "10.2", + "10.2000", + }); + testVariantWith("decimal32", new String[]{"field Variant(String, Decimal32(4))"}, + new Object[]{ + "10.202", + 10.1233d, + }, + new String[]{ + "10.202", + "10.1233", + }); } - public void testVariantWithDateTime() throws Exception { + @Test(groups = {"integration"}, enabled = false) + public void testVariantWithArrays() throws Exception { + // TODO: writing array would need custom serialization logic + testVariantWith("arrays", new String[]{"field Variant(String, Array(String))"}, + new Object[]{ + "a,b", + new String[]{"a", "b"} + }, + new String[]{ + "a,b", + "a,b", + }); + } + @Test(groups = {"integration"}, enabled = false) + public void testVariantWithMaps() throws Exception { + //TODO: similar to arrays } - public void testVariantWithNullable() throws Exception { + @Test(groups = {"integration"}) + public void testVariantWithEnums() throws Exception { + testVariantWith("enums", new String[]{"field Variant(Bool, Enum('stopped' = 1, 'running' = 2))"}, + new Object[]{ + "stopped", + 1, + "running", + 2, + true, + false + }, + new String[]{ + "stopped", + "stopped", + "running", + "running", + "true", + "false" + }); + } + @Test(groups = {"integration"}, enabled = false) + public void testVariantWithTuple() throws Exception { + // TODO: same as array + testVariantWith("arrays", new String[]{"field Variant(String, Tuple(Int32, Float32))"}, + new Object[]{ + "10,0.34", + new Object[] { 10, 0.34f} + }, + new String[]{ + "10,0.34", + "(10,0.34)", + }); } - public void testVariantWithArrays() throws Exception { + private void testVariantWith(String withWhat, String[] fields, Object[] values, String[] expectedStrValues) throws Exception { + String table = "test_variant_with_" + withWhat; + String[] actualFields = new String[fields.length + 1]; + actualFields[0] = "rowId Int32"; + System.arraycopy(fields, 0, actualFields, 1, fields.length); + client.execute("DROP TABLE IF EXISTS " + table).get(); + client.execute(tableDefinition(table, actualFields), (CommandSettings) new CommandSettings().serverSetting("enable_variant_type", "1")).get(); - } + client.register(DTOForVariantPrimitivesTests.class, client.getTableSchema(table)); - public void testVariantWithMaps() throws Exception { + List data = new ArrayList<>(); + for (int i = 0; i < values.length; i++) { + data.add(new DTOForVariantPrimitivesTests(i, values[i])); + } + client.insert(table, data).get().close(); + List rows = client.queryAll("SELECT * FROM " + table); + for (GenericRecord row : rows) { + System.out.println("> " + row.getString("field")); + Assert.assertEquals(row.getString("field"), expectedStrValues[row.getInteger("rowId")]); + } } public static String tableDefinition(String table, String... columns) { diff --git a/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java b/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java index acfce3f50..2d7d1eb04 100644 --- a/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java @@ -1113,7 +1113,6 @@ public void testEnums() { "max_enum8 Enum8('value1' = 1, 'value2' = 2, 'value3' = 127)" ); - final UUID providedUUID = UUID.randomUUID(); final List> valueGenerators = Arrays.asList( () -> "'value1'", () -> "'value3'", @@ -1126,21 +1125,26 @@ public void testEnums() { Assert.assertTrue(r.hasValue("min_enum16"), "No value for column min_enum16 found"); Assert.assertEquals(r.getEnum16("min_enum16"), (short) -32768); Assert.assertEquals(r.getEnum16(1), (short) -32768); + Assert.assertEquals(r.getString(1), "value1"); + }); verifiers.add(r -> { Assert.assertTrue(r.hasValue("max_enum16"), "No value for column max_enum16 found"); Assert.assertEquals(r.getEnum16("max_enum16"), (short) 32767); Assert.assertEquals(r.getEnum16(2), (short) 32767); + Assert.assertEquals(r.getString(2), "value3"); }); verifiers.add(r -> { Assert.assertTrue(r.hasValue("min_enum8"), "No value for column min_enum8 found"); Assert.assertEquals(r.getEnum8("min_enum8"), (byte) -128); Assert.assertEquals(r.getEnum8(3), (byte) -128); + Assert.assertEquals(r.getString(3), "value1"); }); verifiers.add(r -> { Assert.assertTrue(r.hasValue("max_enum8"), "No value for column max_enum8 found"); Assert.assertEquals(r.getEnum8("max_enum8"), (byte) 127); Assert.assertEquals(r.getEnum8(4), (byte) 127); + Assert.assertEquals(r.getString(4), "value3"); }); testDataTypes(columns, valueGenerators, verifiers); From 14fbca6e76b5908c169fab38f88e165ff164109b Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Sat, 25 Jan 2025 07:50:06 -0800 Subject: [PATCH 5/7] implemented support of arrays with different depth --- .../com/clickhouse/data/ClickHouseColumn.java | 37 +++++- .../clickhouse/data/ClickHouseDataType.java | 5 +- .../com/clickhouse/client/api/Client.java | 8 +- .../internal/AbstractBinaryFormatReader.java | 4 +- .../internal/BinaryStreamReader.java | 3 + ...TypeTests.java => DataTypeUtilsTests.java} | 2 +- .../client/datatypes/DataTypeTests.java | 118 ++++++++++++++---- .../client/datatypes/NestedTypesDTO.java | 2 +- .../clickhouse/client/query/QueryTests.java | 1 - 9 files changed, 150 insertions(+), 30 deletions(-) rename client-v2/src/test/java/com/clickhouse/client/api/{DataTypeTests.java => DataTypeUtilsTests.java} (94%) diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java index 48b70ee06..58f096cd6 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java @@ -41,6 +41,7 @@ import java.time.OffsetDateTime; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -48,6 +49,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.TimeZone; /** @@ -99,6 +101,8 @@ public final class ClickHouseColumn implements Serializable { private Map, Integer> classToVariantOrdNumMap; + private Map, Integer> arrayToVariantOrdNumMap; + private static ClickHouseColumn update(ClickHouseColumn column) { column.enumConstants = ClickHouseEnum.EMPTY; int size = column.parameters.size(); @@ -446,6 +450,21 @@ protected static int readColumn(String args, int startIndex, int len, String nam } } column.classToVariantOrdNumMap = ClickHouseDataType.buildVariantMapping(variantDataTypes); + + for (int ordNum = 0; ordNum < nestedColumns.size(); ordNum++) { + ClickHouseColumn nestedColumn = nestedColumns.get(ordNum); + if (nestedColumn.getDataType() == ClickHouseDataType.Array) { + Set> classSet = ClickHouseDataType.DATA_TYPE_TO_CLASS.get(nestedColumn.arrayBaseColumn.dataType); + if (classSet != null) { + if (column.arrayToVariantOrdNumMap == null) { + column.arrayToVariantOrdNumMap = new HashMap<>(); + } + for (Class c : classSet) { + column.arrayToVariantOrdNumMap.put(c, ordNum); + } + } + } + } } if (column == null) { @@ -650,7 +669,23 @@ public boolean isAggregateFunction() { } public int getVariantOrdNum(Object value) { - return classToVariantOrdNumMap.getOrDefault(value.getClass(), -1); + if (value != null && value.getClass().isArray()) { + Class c = value.getClass(); + while (c.isArray()) { + c = c.getComponentType(); + } + return arrayToVariantOrdNumMap.getOrDefault(c, -1); + } else if (value != null && value instanceof List) { + Object tmpV = ((List)value).get(0); + Class valueClass = tmpV.getClass(); + while (tmpV instanceof List) { + tmpV = ((List)tmpV).get(0); + valueClass = tmpV.getClass(); + } + return arrayToVariantOrdNumMap.getOrDefault(valueClass, -1); + } else { + return classToVariantOrdNumMap.getOrDefault(value.getClass(), -1); + } } public boolean isArray() { 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 d2515ea3f..49ca5cf7d 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -151,15 +151,15 @@ public static Map, Integer> buildVariantMapping(List entry : intTypesMappings.entrySet()) { DATA_TYPE_TO_CLASS.get(entry.getKey()).forEach(c -> variantMapping.put(c, entry.getValue())); } + // add decimals for (java.util.Map.Entry entry : decTypesMappings.entrySet()) { DATA_TYPE_TO_CLASS.get(entry.getKey()).forEach(c -> variantMapping.put(c, entry.getValue())); } - return variantMapping; } @@ -211,6 +211,7 @@ static Map>> dataTypeClassMap() { map.put(Enum8, setOf(java.lang.String.class,byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class)); map.put(Enum16, setOf(java.lang.String.class,byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class)); + map.put(Array, setOf(List.class, Object[].class, byte[].class, short[].class, int[].class, long[].class, boolean[].class)); return map; } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java index 613257036..f936119ee 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java @@ -1807,7 +1807,13 @@ public List queryAll(String sqlQuery, Class clazz, TableSchema schema, (RowBinaryWithNamesAndTypesFormatReader) newBinaryFormatReader(response); while (true) { - Object record = allocator == null ? clazz.getDeclaredConstructor().newInstance() : allocator.get(); + + Object record; + try { + record = allocator == null ? clazz.getDeclaredConstructor().newInstance() : allocator.get(); + } catch (NoSuchMethodException e) { + throw new ClientException("Failed to instantiate DTO to store data: no-args constructor is not defined"); + } if (reader.readToPOJO(classDeserializers, record)) { records.add((T) record); } else { diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java index b1e6be534..cfbe233cc 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java @@ -109,7 +109,7 @@ public boolean readToPOJO(Map deserializers, Object obj ) th } throw e; } catch (Exception e) { - throw new ClientException("Failed to put value of '" + column.getColumnName() + "' into POJO", e); + throw new ClientException("Failed to set value of '" + column.getColumnName(), e); } } return true; @@ -303,6 +303,8 @@ public static String readAsString(Object value, ClickHouseColumn column) { } else if (dataType == ClickHouseDataType.Enum8 || dataType == ClickHouseDataType.Enum16) { return column.getEnumConstants().name(num); } + } else if (value instanceof BinaryStreamReader.ArrayValue) { + return ((BinaryStreamReader.ArrayValue)value).asList().toString(); } return value.toString(); } 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 7d5cacc96..5233d6ec1 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 @@ -247,6 +247,9 @@ private static T convertArray(ArrayValue value, Class typeHint) { if (typeHint.isAssignableFrom(List.class)) { return (T) value.asList(); } + if (typeHint.isArray()) { + return (T) value.array; + } return (T) value; } diff --git a/client-v2/src/test/java/com/clickhouse/client/api/DataTypeTests.java b/client-v2/src/test/java/com/clickhouse/client/api/DataTypeUtilsTests.java similarity index 94% rename from client-v2/src/test/java/com/clickhouse/client/api/DataTypeTests.java rename to client-v2/src/test/java/com/clickhouse/client/api/DataTypeUtilsTests.java index 7987ae8ed..4c5a9e70c 100644 --- a/client-v2/src/test/java/com/clickhouse/client/api/DataTypeTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/api/DataTypeUtilsTests.java @@ -6,7 +6,7 @@ import static org.testng.AssertJUnit.assertEquals; -public class DataTypeTests { +public class DataTypeUtilsTests { @Test 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 3d1600748..4440df509 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 @@ -5,13 +5,16 @@ import com.clickhouse.client.ClickHouseProtocol; import com.clickhouse.client.api.Client; import com.clickhouse.client.api.command.CommandSettings; +import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader; import com.clickhouse.client.api.enums.Protocol; import com.clickhouse.client.api.insert.InsertResponse; import com.clickhouse.client.api.insert.InsertSettings; +import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.GenericRecord; import com.clickhouse.data.ClickHouseDataType; import lombok.AllArgsConstructor; import lombok.Data; +import lombok.NoArgsConstructor; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -25,6 +28,9 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; +import java.util.function.Consumer; public class DataTypeTests extends BaseIntegrationTest { @@ -59,39 +65,83 @@ public void setUp() throws IOException { .build(); } - @AfterMethod(groups = { "integration" }) + @AfterMethod(groups = {"integration"}) public void tearDown() { client.close(); } + private void writeReadVerify(String table, String tableDef, Class dtoClass, List data, + BiConsumer, T> rowVerifier) throws Exception { + client.execute("DROP TABLE IF EXISTS " + table).get(); + client.execute(tableDef); + + final TableSchema tableSchema = client.getTableSchema(table); + client.register(dtoClass, tableSchema); + client.insert(table, data); + final AtomicInteger rowCount = new AtomicInteger(0); + client.queryAll("SELECT * FROM " + table, dtoClass, tableSchema).forEach(dto -> { + rowVerifier.accept(data, dto); + rowCount.incrementAndGet(); + }); + + Assert.assertEquals(rowCount.get(), data.size()); + } @Test(groups = {"integration"}) public void testNestedDataTypes() throws Exception { final String table = "test_nested_types"; - String tblCreateSQL = NestedTypesDTO.tblCreateSQL(table); - client.execute("DROP TABLE IF EXISTS " + table).get(); - client.execute(tblCreateSQL); + writeReadVerify(table, + NestedTypesDTO.tblCreateSQL(table), + NestedTypesDTO.class, + Arrays.asList(new NestedTypesDTO(0, new Object[]{(short) 127, "test 1"}, new double[]{0.3d, 0.4d})), + (data, dto) -> { + NestedTypesDTO dataDto = data.get(dto.getRowId()); + Assert.assertEquals(dto.getTuple1(), dataDto.getTuple1()); + Assert.assertEquals(dto.getPoint1(), dataDto.getPoint1()); + }); + } - client.register(NestedTypesDTO.class, client.getTableSchema(table)); + @Test(groups = {"integration"}) + public void testArrays() throws Exception { + final String table = "test_arrays"; + writeReadVerify(table, + DTOForArraysTests.tblCreateSQL(table), + DTOForArraysTests.class, + Arrays.asList(new DTOForArraysTests( + 0, Arrays.asList("db", "fast"), new int[]{1, 2, 3}, new String[]{"a", "b", "c"})), + (data, dto) -> { + DTOForArraysTests dataDto = data.get(dto.getRowId()); + System.out.println(dto.getWords()); + Assert.assertEquals(dto.getWords(), dataDto.getWords()); + System.out.println(Arrays.asList(dto.getLetters())); + Assert.assertEquals(dto.getLetters(), dataDto.getLetters()); + System.out.println(Arrays.asList(dto.getNumbers())); + Assert.assertEquals(dto.getNumbers(), dataDto.getNumbers()); + }); + } - List data = - Arrays.asList(new NestedTypesDTO(0, new Object[] {(short)127, "test 1"}, new Double[] {0.3d, 0.4d} )); - client.insert(table, data); + @Data + @AllArgsConstructor + @NoArgsConstructor + public static class DTOForArraysTests { + private int rowId; - List rows = client.queryAll("SELECT * FROM " + table); - for (GenericRecord row : rows) { - NestedTypesDTO dto = data.get(row.getInteger("rowId")); - Assert.assertEquals(row.getTuple("tuple1"), dto.getTuple1()); - Assert.assertEquals(row.getGeoPoint("point1").getValue(), dto.getPoint1()); - } + private List words; + + private int[] numbers; + private String[] letters; + + public static String tblCreateSQL(String table) { + return tableDefinition(table, "rowId Int16", "words Array(String)", "numbers Array(Int32)", + "letters Array(String)"); + } } @Test(groups = {"integration"}) public void testVariantWithSimpleDataTypes() throws Exception { final String table = "test_variant_primitives"; final DataTypesTestingPOJO sample = new DataTypesTestingPOJO(); - System.out.println("sample: " + sample); dataTypesLoop: for (ClickHouseDataType dataType : ClickHouseDataType.values()) { @@ -168,7 +218,7 @@ public void testVariantWithSimpleDataTypes() throws Exception { case DateTime: case DateTime32: strValue = row.getLocalDateTime("field").truncatedTo(ChronoUnit.SECONDS).toString(); - value = ((LocalDateTime)value ).truncatedTo(ChronoUnit.SECONDS).toString(); + value = ((LocalDateTime) value).truncatedTo(ChronoUnit.SECONDS).toString(); break; case Point: strValue = row.getGeoPoint("field").toString(); @@ -183,7 +233,7 @@ public void testVariantWithSimpleDataTypes() throws Exception { strValue = row.getGeoMultiPolygon("field").toString(); break; } - System.out.println("field: " + strValue + " value " + value); + System.out.println("field: " + strValue + " value " + value); if (value.getClass().isPrimitive()) { Assert.assertEquals(strValue, String.valueOf(value)); } else { @@ -222,17 +272,41 @@ public void testVariantWithDecimals() throws Exception { }); } - @Test(groups = {"integration"}, enabled = false) + @Test(groups = {"integration"}) public void testVariantWithArrays() throws Exception { - // TODO: writing array would need custom serialization logic testVariantWith("arrays", new String[]{"field Variant(String, Array(String))"}, new Object[]{ "a,b", - new String[]{"a", "b"} + new String[]{"a", "b"}, + Arrays.asList("c", "d") }, new String[]{ "a,b", - "a,b", + "[a, b]", + "[c, d]" + }); + testVariantWith("arrays", new String[]{"field Variant(Array(String), Array(Int32))"}, + new Object[]{ + new int[]{1, 2}, + new String[]{"a", "b"}, + Arrays.asList("c", "d") + }, + new String[]{ + "[1, 2]", + "[a, b]", + "[c, d]", + }); + + testVariantWith("arrays", new String[]{"field Variant(Array(Array(String)), Array(Array(Int32)))"}, + new Object[]{ + new int[][]{ new int[] {1, 2}, new int[] { 3, 4}}, + new String[][]{new String[]{"a", "b"}, new String[]{"c", "d"}}, +// Arrays.asList(Arrays.asList("e", "f"), Arrays.asList("j", "h")) + }, + new String[]{ + "[[1, 2], [3, 4]]", + "[[a, b], [c, d]]", +// "[c, d]", }); } @@ -268,7 +342,7 @@ public void testVariantWithTuple() throws Exception { testVariantWith("arrays", new String[]{"field Variant(String, Tuple(Int32, Float32))"}, new Object[]{ "10,0.34", - new Object[] { 10, 0.34f} + new Object[]{10, 0.34f} }, new String[]{ "10,0.34", diff --git a/client-v2/src/test/java/com/clickhouse/client/datatypes/NestedTypesDTO.java b/client-v2/src/test/java/com/clickhouse/client/datatypes/NestedTypesDTO.java index 5f5800405..88552b7f0 100644 --- a/client-v2/src/test/java/com/clickhouse/client/datatypes/NestedTypesDTO.java +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/NestedTypesDTO.java @@ -17,7 +17,7 @@ public class NestedTypesDTO { private Object[] tuple1; - private Object[] point1; + private double[] point1; public static String tblCreateSQL(String table) { return tableDefinition(table, diff --git a/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java b/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java index 2d7d1eb04..8c9b8c3d5 100644 --- a/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java @@ -1149,7 +1149,6 @@ public void testEnums() { testDataTypes(columns, valueGenerators, verifiers); } - @Test public void testUUID() { final List columns = Arrays.asList( From 2e0eb4f091596807f6fc45c242e84cd8a8a1953f Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 28 Jan 2025 14:23:15 -0800 Subject: [PATCH 6/7] implemented map for variant --- .../com/clickhouse/data/ClickHouseColumn.java | 51 ++++++++++++++++++- .../internal/SerializerUtils.java | 2 +- .../client/datatypes/DataTypeTests.java | 40 +++++++++++++-- 3 files changed, 86 insertions(+), 7 deletions(-) diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java index 58f096cd6..a65dbda74 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java @@ -103,6 +103,10 @@ public final class ClickHouseColumn implements Serializable { private Map, Integer> arrayToVariantOrdNumMap; + private Map, Integer> mapKeyToVariantOrdNumMap; + private Map, Integer> mapValueToVariantOrdNumMap; + + private static ClickHouseColumn update(ClickHouseColumn column) { column.enumConstants = ClickHouseEnum.EMPTY; int size = column.parameters.size(); @@ -463,6 +467,23 @@ protected static int readColumn(String args, int startIndex, int len, String nam column.arrayToVariantOrdNumMap.put(c, ordNum); } } + } else if (nestedColumn.getDataType() == ClickHouseDataType.Map) { + Set> keyClassSet = ClickHouseDataType.DATA_TYPE_TO_CLASS.get(nestedColumn.getKeyInfo().getDataType()); + Set> valueClassSet = ClickHouseDataType.DATA_TYPE_TO_CLASS.get(nestedColumn.getValueInfo().getDataType()); + if (keyClassSet != null && valueClassSet != null) { + if (column.mapKeyToVariantOrdNumMap == null) { + column.mapKeyToVariantOrdNumMap = new HashMap<>(); + } + if (column.mapValueToVariantOrdNumMap == null) { + column.mapValueToVariantOrdNumMap = new HashMap<>(); + } + for (Class c : keyClassSet) { + column.mapKeyToVariantOrdNumMap.put(c, ordNum); + } + for (Class c : valueClassSet) { + column.mapValueToVariantOrdNumMap.put(c, ordNum); + } + } } } } @@ -670,19 +691,45 @@ public boolean isAggregateFunction() { public int getVariantOrdNum(Object value) { if (value != null && value.getClass().isArray()) { + // TODO: add cache by value class Class c = value.getClass(); while (c.isArray()) { c = c.getComponentType(); } return arrayToVariantOrdNumMap.getOrDefault(c, -1); } else if (value != null && value instanceof List) { - Object tmpV = ((List)value).get(0); + // TODO: add cache by instance of the list + Object tmpV = ((List) value).get(0); Class valueClass = tmpV.getClass(); while (tmpV instanceof List) { - tmpV = ((List)tmpV).get(0); + tmpV = ((List) tmpV).get(0); valueClass = tmpV.getClass(); } return arrayToVariantOrdNumMap.getOrDefault(valueClass, -1); + } else if (value != null && value instanceof Map) { + // TODO: add cache by instance of map + Map map = (Map) value; + if (!map.isEmpty()) { + for (Map.Entry e : map.entrySet()) { + if (e.getValue() != null) { + int keyOrdNum = mapKeyToVariantOrdNumMap.getOrDefault(e.getKey().getClass(), -1); + int valueOrdNum = mapValueToVariantOrdNumMap.getOrDefault(e.getValue().getClass(), -1); + + if (keyOrdNum == valueOrdNum) { + return valueOrdNum; // exact match + } else if (keyOrdNum != -1 && valueOrdNum != -1) { + if (ClickHouseDataType.DATA_TYPE_TO_CLASS.get(nested.get(keyOrdNum).getValueInfo().getDataType()).contains(e.getValue().getClass())){ + return keyOrdNum; // can write to map found by key class because values are compatible + } else { + return valueOrdNum; + } + } + + break; + } + } + } + return -1; } else { return classToVariantOrdNumMap.getOrDefault(value.getClass(), -1); } 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 06bc84abf..3e6b6ba71 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 @@ -112,7 +112,7 @@ private static void serializeArrayData(OutputStream stream, Object value, ClickH } writeNonNull(stream); } - serializeData(stream, val, column.getArrayBaseColumn()); + serializeData(stream, val, column.getNestedColumns().get(0)); } } else if (value.getClass().isArray()) { writeVarInt(stream, Array.getLength(value)); 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 4440df509..bbb5fa02d 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 @@ -27,7 +27,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; import java.util.function.Consumer; @@ -301,18 +303,48 @@ public void testVariantWithArrays() throws Exception { new Object[]{ new int[][]{ new int[] {1, 2}, new int[] { 3, 4}}, new String[][]{new String[]{"a", "b"}, new String[]{"c", "d"}}, -// Arrays.asList(Arrays.asList("e", "f"), Arrays.asList("j", "h")) + Arrays.asList(Arrays.asList("e", "f"), Arrays.asList("j", "h")) }, new String[]{ "[[1, 2], [3, 4]]", "[[a, b], [c, d]]", -// "[c, d]", + "[[e, f], [j, h]]", }); } - @Test(groups = {"integration"}, enabled = false) + @Test(groups = {"integration"}) public void testVariantWithMaps() throws Exception { - //TODO: similar to arrays + Map map1 = new HashMap<>(); + map1.put("key1", (byte) 1); + map1.put("key2", (byte) 2); + map1.put("key3", (byte) 3); + + testVariantWith("maps", new String[]{"field Variant(Map(String, String), Map(String, Int128))"}, + new Object[]{ + map1 + }, + new String[]{ + "{key1=1, key2=2, key3=3}", + }); + + + Map map2 = new HashMap<>(); + map2.put(1, "a"); + map2.put(2, "b"); + + Map map3 = new HashMap<>(); + map3.put("1", "a"); + map3.put("2", "b"); + + testVariantWith("maps", new String[]{"field Variant(Map(Int32, String), Map(String, String))"}, + new Object[]{ + map2, + map3 + }, + new String[]{ + "{1=a, 2=b}", + "{1=a, 2=b}", + }); } @Test(groups = {"integration"}) From a34707e75526183d78dc374d23a65e9205680ac0 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 28 Jan 2025 22:25:54 -0800 Subject: [PATCH 7/7] disabled old JDBC test --- .../src/main/java/com/clickhouse/data/ClickHouseColumn.java | 2 +- .../java/com/clickhouse/jdbc/ClickHouseStatementTest.java | 4 +--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java index a65dbda74..d723c65d6 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java @@ -707,7 +707,7 @@ public int getVariantOrdNum(Object value) { } return arrayToVariantOrdNumMap.getOrDefault(valueClass, -1); } else if (value != null && value instanceof Map) { - // TODO: add cache by instance of map + // TODO: add cache by instance of map Map map = (Map) value; if (!map.isEmpty()) { for (Map.Entry e : map.entrySet()) { diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java index c7ca5eca9..954db49c9 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java @@ -1528,7 +1528,7 @@ public void testMaxResultsRows() throws SQLException { } } - @Test(groups = "integration") + @Test(groups = "integration", enabled = false) public void testVariantDataType() throws SQLException { String table = "test_variant_type_01"; Properties props = new Properties(); @@ -1548,9 +1548,7 @@ public void testVariantDataType() throws SQLException { while (rs.next()) { Object variantValue = rs.getObject(1); Object name = rs.getString(2); - Object variantSubColumn = rs.getObject("v.String"); System.out.println("-> " + name + " : " + variantValue); - System.out.println("sub: " + variantSubColumn); } } }