From f0645b17f80ee786192d1fd80eb15168571db6e4 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Mon, 3 Feb 2025 14:32:58 -0800 Subject: [PATCH 01/21] added basic dynamic support for rowbinary reader --- .../clickhouse/data/ClickHouseDataType.java | 212 +++++++++++++----- .../internal/BinaryStreamReader.java | 95 ++++++-- .../clickhouse/client/insert/InsertTests.java | 1 + .../clickhouse/client/query/QueryTests.java | 19 +- 4 files changed, 253 insertions(+), 74 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 9c077f7c0..ffc3f8bc2 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -36,72 +36,126 @@ */ @SuppressWarnings("squid:S115") public enum ClickHouseDataType { - Bool(Boolean.class, false, false, true, 1, 1, 0, 0, 0, false, "BOOLEAN"), - Date(LocalDate.class, false, false, false, 2, 10, 0, 0, 0, false), - Date32(LocalDate.class, false, false, false, 4, 10, 0, 0, 0, false), - DateTime(LocalDateTime.class, true, false, false, 0, 29, 0, 0, 9, false, "TIMESTAMP"), - DateTime32(LocalDateTime.class, true, false, false, 4, 19, 0, 0, 0, false), - DateTime64(LocalDateTime.class, true, false, false, 8, 29, 3, 0, 9, false), - Enum8(String.class, true, true, false, 1, 0, 0, 0, 0, false, "ENUM"), - Enum16(String.class, true, true, false, 2, 0, 0, 0, 0, false), - FixedString(String.class, true, true, false, 0, 0, 0, 0, 0, false, "BINARY"), - Int8(Byte.class, false, true, true, 1, 3, 0, 0, 0, false, "BYTE", "INT1", "INT1 SIGNED", "TINYINT", + Bool(Boolean.class, false, false, true, 1, 1, 0, 0, 0, false,0x2D, "BOOLEAN"), + Date(LocalDate.class, false, false, false, 2, 10, 0, 0, 0, false, 0x0F), + Date32(LocalDate.class, false, false, false, 4, 10, 0, 0, 0, false, 0x10), + DateTime(LocalDateTime.class, true, false, false, 0, 29, 0, 0, 9, false, 0x11, "TIMESTAMP"), + DateTime32(LocalDateTime.class, true, false, false, 4, 19, 0, 0, 0, false, 0x12), + DateTime64(LocalDateTime.class, true, false, false, 8, 29, 3, 0, 9, false, 0x13), + Enum8(String.class, true, true, false, 1, 0, 0, 0, 0, false, 0x17, "ENUM"), + Enum16(String.class, true, true, false, 2, 0, 0, 0, 0, false, 0x18), + FixedString(String.class, true, true, false, 0, 0, 0, 0, 0, false, 0x16, "BINARY"), + Int8(Byte.class, false, true, true, 1, 3, 0, 0, 0, false, 0x07,"BYTE", "INT1", "INT1 SIGNED", "TINYINT", "TINYINT SIGNED"), - UInt8(UnsignedByte.class, false, true, false, 1, 3, 0, 0, 0, false, "INT1 UNSIGNED", "TINYINT UNSIGNED"), - Int16(Short.class, false, true, true, 2, 5, 0, 0, 0, false, "SMALLINT", "SMALLINT SIGNED"), - UInt16(UnsignedShort.class, false, true, false, 2, 5, 0, 0, 0, false, "SMALLINT UNSIGNED", "YEAR"), - Int32(Integer.class, false, true, true, 4, 10, 0, 0, 0, false, "INT", "INT SIGNED", "INTEGER", "INTEGER SIGNED", + UInt8(UnsignedByte.class, false, true, false, 1, 3, 0, 0, 0, false,0x01, "INT1 UNSIGNED", "TINYINT UNSIGNED"), + Int16(Short.class, false, true, true, 2, 5, 0, 0, 0, false, 0x08,"SMALLINT", "SMALLINT SIGNED"), + UInt16(UnsignedShort.class, false, true, false, 2, 5, 0, 0, 0, false, 0x02,"SMALLINT UNSIGNED", "YEAR"), + Int32(Integer.class, false, true, true, 4, 10, 0, 0, 0, false, 0x09, "INT", "INT SIGNED", "INTEGER", "INTEGER SIGNED", "MEDIUMINT", "MEDIUMINT SIGNED"), // https://docs.oracle.com/javase/tutorial/java/nutsandbolts/datatypes.html#PageTitle - UInt32(UnsignedInteger.class, false, true, false, 4, 10, 0, 0, 0, false, "INT UNSIGNED", "INTEGER UNSIGNED", + UInt32(UnsignedInteger.class, false, true, false, 4, 10, 0, 0, 0, false, 0x03, "INT UNSIGNED", "INTEGER UNSIGNED", "MEDIUMINT UNSIGNED"), - Int64(Long.class, false, true, true, 8, 19, 0, 0, 0, false, "BIGINT", "BIGINT SIGNED", "TIME"), - IntervalYear(Long.class, false, true, true, 8, 19, 0, 0, 0, false), - IntervalQuarter(Long.class, false, true, true, 8, 19, 0, 0, 0, false), - IntervalMonth(Long.class, false, true, true, 8, 19, 0, 0, 0, false), - IntervalWeek(Long.class, false, true, true, 8, 19, 0, 0, 0, false), - IntervalDay(Long.class, false, true, true, 8, 19, 0, 0, 0, false), - IntervalHour(Long.class, false, true, true, 8, 19, 0, 0, 0, false), - IntervalMinute(Long.class, false, true, true, 8, 19, 0, 0, 0, false), - IntervalSecond(Long.class, false, true, true, 8, 19, 0, 0, 0, false), - IntervalMicrosecond(Long.class, false, true, true, 8, 19, 0, 0, 0, false), - IntervalMillisecond(Long.class, false, true, true, 8, 19, 0, 0, 0, false), - IntervalNanosecond(Long.class, false, true, true, 8, 19, 0, 0, 0, false), - UInt64(UnsignedLong.class, false, true, false, 8, 20, 0, 0, 0, false, "BIGINT UNSIGNED", "BIT", "SET"), - Int128(BigInteger.class, false, true, true, 16, 39, 0, 0, 0, false), - UInt128(BigInteger.class, false, true, false, 16, 39, 0, 0, 0, false), - Int256(BigInteger.class, false, true, true, 32, 77, 0, 0, 0, false), - UInt256(BigInteger.class, false, true, false, 32, 78, 0, 0, 0, false), + Int64(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x0A,"BIGINT", "BIGINT SIGNED", "TIME"), + IntervalYear(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22), + IntervalQuarter(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22), + IntervalMonth(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22), + IntervalWeek(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22), + IntervalDay(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22), + IntervalHour(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22), + IntervalMinute(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22), + IntervalSecond(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22), + IntervalMicrosecond(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22), + IntervalMillisecond(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22), + IntervalNanosecond(Long.class, false, true, true, 8, 19, 0, 0, 0, false, 0x22), + UInt64(UnsignedLong.class, false, true, false, 8, 20, 0, 0, 0, false, 0x04, "BIGINT UNSIGNED", "BIT", "SET"), + Int128(BigInteger.class, false, true, true, 16, 39, 0, 0, 0, false, 0x0B), + UInt128(BigInteger.class, false, true, false, 16, 39, 0, 0, 0, false, 0x05), + Int256(BigInteger.class, false, true, true, 32, 77, 0, 0, 0, false, 0x0C), + UInt256(BigInteger.class, false, true, false, 32, 78, 0, 0, 0, false, 0x06), Decimal(BigDecimal.class, true, false, true, 0, 76, 0, 0, 76, false, "DEC", "FIXED", "NUMERIC"), - Decimal32(BigDecimal.class, true, false, true, 4, 9, 9, 0, 9, false), - Decimal64(BigDecimal.class, true, false, true, 8, 18, 18, 0, 18, false), - Decimal128(BigDecimal.class, true, false, true, 16, 38, 38, 0, 38, false), - Decimal256(BigDecimal.class, true, false, true, 32, 76, 20, 0, 76, false), - Float32(Float.class, false, true, true, 4, 12, 0, 0, 38, false, "FLOAT", "REAL", "SINGLE"), - Float64(Double.class, false, true, true, 8, 22, 0, 0, 308, false, "DOUBLE", "DOUBLE PRECISION"), - IPv4(Inet4Address.class, false, true, false, 4, 10, 0, 0, 0, false, "INET4"), - IPv6(Inet6Address.class, false, true, false, 16, 39, 0, 0, 0, false, "INET6"), - UUID(UUID.class, false, true, false, 16, 69, 0, 0, 0, false), - Point(Object.class, false, true, true, 33, 0, 0, 0, 0, true), // same as Tuple(Float64, Float64) - Polygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Ring) - MultiPolygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Polygon) - Ring(Object.class, false, true, true, 0, 0, 0, 0, 0, true), // same as Array(Point) - JSON(Object.class, false, false, false, 0, 0, 0, 0, 0, true), // same as Object('JSON') + Decimal32(BigDecimal.class, true, false, true, 4, 9, 9, 0, 9, false, 0x19), + Decimal64(BigDecimal.class, true, false, true, 8, 18, 18, 0, 18, false, 0x1A), + Decimal128(BigDecimal.class, true, false, true, 16, 38, 38, 0, 38, false, 0x1B), + Decimal256(BigDecimal.class, true, false, true, 32, 76, 20, 0, 76, false, 0x1C), + Float32(Float.class, false, true, true, 4, 12, 0, 0, 38, false, 0x0D, "FLOAT", "REAL", "SINGLE"), + Float64(Double.class, false, true, true, 8, 22, 0, 0, 308, false, 0x0E, "DOUBLE", "DOUBLE PRECISION"), + IPv4(Inet4Address.class, false, true, false, 4, 10, 0, 0, 0, false, 0x28, "INET4"), + IPv6(Inet6Address.class, false, true, false, 16, 39, 0, 0, 0, false, 0x29, "INET6"), + UUID(UUID.class, false, true, false, 16, 69, 0, 0, 0, false, 0x1D), + Point(Object.class, false, true, true, 33, 0, 0, 0, 0, true, 0x2C), // same as Tuple(Float64, Float64) + Polygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Ring) + MultiPolygon(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Polygon) + Ring(Object.class, false, true, true, 0, 0, 0, 0, 0, true, 0x2C), // same as Array(Point) + JSON(Object.class, false, false, false, 0, 0, 0, 0, 0, true, 0x30), + @Deprecated Object(Object.class, true, true, false, 0, 0, 0, 0, 0, true), - String(String.class, false, false, false, 0, 0, 0, 0, 0, false, "BINARY LARGE OBJECT", "BINARY VARYING", "BLOB", + String(String.class, false, false, false, 0, 0, 0, 0, 0, false, 0x15, "BINARY LARGE OBJECT", "BINARY VARYING", "BLOB", "BYTEA", "CHAR", "CHAR LARGE OBJECT", "CHAR VARYING", "CHARACTER", "CHARACTER LARGE OBJECT", "CHARACTER VARYING", "CLOB", "GEOMETRY", "LONGBLOB", "LONGTEXT", "MEDIUMBLOB", "MEDIUMTEXT", "NATIONAL CHAR", "NATIONAL CHAR VARYING", "NATIONAL CHARACTER", "NATIONAL CHARACTER LARGE OBJECT", "NATIONAL CHARACTER VARYING", "NCHAR", "NCHAR LARGE OBJECT", "NCHAR VARYING", "NVARCHAR", "TEXT", "TINYBLOB", "TINYTEXT", "VARBINARY", "VARCHAR", "VARCHAR2"), - Array(Object.class, true, true, false, 0, 0, 0, 0, 0, true), - Map(Map.class, true, true, false, 0, 0, 0, 0, 0, true), - Nested(Object.class, true, true, false, 0, 0, 0, 0, 0, true), - Tuple(List.class, true, true, false, 0, 0, 0, 0, 0, true), - Nothing(Object.class, false, true, false, 0, 0, 0, 0, 0, true), - SimpleAggregateFunction(String.class, true, true, false, 0, 0, 0, 0, 0, false), + Array(Object.class, true, true, false, 0, 0, 0, 0, 0, true, 0x1E), + Map(Map.class, true, true, false, 0, 0, 0, 0, 0, true, 0x27), + Nested(Object.class, true, true, false, 0, 0, 0, 0, 0, true, 0x2F), + Tuple(List.class, true, true, false, 0, 0, 0, 0, 0, true, 0x1F), + Nothing(Object.class, false, true, false, 0, 0, 0, 0, 0, true, 0x00), + SimpleAggregateFunction(String.class, true, true, false, 0, 0, 0, 0, 0, false, 0x2E), // 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, 0x25), + Dynamic(Object.class, true, true, false, 0, 0, 0, 0, 0, true, 0x2B), + ; + + + public static final byte INTERVAL_BIN_TAG = 0x22; + + public static final byte NULLABLE_BIN_TAG = 0x23; + + public static final byte LOW_CARDINALITY_BIN_TAG = 0x26; + + public static final byte SET_BIN_TAG = 0x21; + + public enum IntervalKindBinTag { + Nanosecond(IntervalNanosecond, 0x00), + Microsecond(IntervalMicrosecond, 0x01), + + Millisecond(IntervalMillisecond, 0x02), + + Second(IntervalSecond, 0x03), + + Minute(IntervalMinute, 0x04), + + Hour(IntervalHour, 0x05), + + Day(IntervalDay, 0x06), + + Week(IntervalWeek, 0x07), + + Month(IntervalMonth, 0x08), + + Quarter(IntervalQuarter, 0x09), + + Year(IntervalYear, 0x1A) // why 1A ? + + ; + + private ClickHouseDataType intervalType; + + byte tag; + IntervalKindBinTag(ClickHouseDataType clickHouseDataType, int tag) { + this.intervalType = clickHouseDataType; + this.tag = (byte) tag; + } + + public ClickHouseDataType getIntervalType() { + return intervalType; + } + + public byte getTag() { + return tag; + } + } + /** * Immutable set(sorted) for all aliases. @@ -113,6 +167,10 @@ public enum ClickHouseDataType { */ public static final Map name2type; + public static final Map binTag2Type; + + public static final Map intervalKind2Type; + static { Set set = new TreeSet<>(); Map map = new HashMap<>(); @@ -141,6 +199,18 @@ public enum ClickHouseDataType { allAliases = Collections.unmodifiableSet(set); name2type = Collections.unmodifiableMap(map); + + Map tmpbinTag2Type = new HashMap<>(); + for (ClickHouseDataType type : ClickHouseDataType.values()) { + tmpbinTag2Type.put((byte) type.getBinTag(), type); + } + binTag2Type = Collections.unmodifiableMap(tmpbinTag2Type); + + Map tmpIntervalKind2Type = new HashMap<>(); + for (IntervalKindBinTag kind : IntervalKindBinTag.values()) { + tmpIntervalKind2Type.put(kind.getTag(), kind.getIntervalType()); + } + intervalKind2Type = Collections.unmodifiableMap(tmpIntervalKind2Type); } /** @@ -373,6 +443,8 @@ public static Class toWiderPrimitiveType(Class javaClass) { private final int maxScale; private final boolean nestedType; + private final byte binTag; + /** * Default constructor. * @@ -404,6 +476,30 @@ public static Class toWiderPrimitiveType(Class javaClass) { this.minScale = minScale; this.maxScale = maxScale; this.nestedType = nestedType; + this.binTag = -1; + if (aliases == null || aliases.length == 0) { + this.aliases = Collections.emptyList(); + } else { + this.aliases = Collections.unmodifiableList(Arrays.asList(aliases)); + } + } + + ClickHouseDataType(Class javaClass, boolean parameter, boolean caseSensitive, boolean signed, int byteLength, + int maxPrecision, int defaultScale, int minScale, int maxScale, boolean nestedType, int binTag, String... aliases) { + this.objectType = toObjectType(javaClass); + this.widerObjectType = !signed ? toWiderObjectType(javaClass) : this.objectType; + this.primitiveType = toPrimitiveType(javaClass); + this.widerPrimitiveType = !signed ? toWiderPrimitiveType(javaClass) : this.primitiveType; + this.parameter = parameter; + this.caseSensitive = caseSensitive; + this.signed = signed; + this.byteLength = byteLength; + this.maxPrecision = maxPrecision; + this.defaultScale = defaultScale; + this.minScale = minScale; + this.maxScale = maxScale; + this.nestedType = nestedType; + this.binTag = (byte) binTag; if (aliases == null || aliases.length == 0) { this.aliases = Collections.emptyList(); } else { @@ -541,4 +637,12 @@ public int getMinScale() { public int getMaxScale() { return maxScale; } + + /** + * Returns a binary tag for the type + * @return tag value + */ + public byte getBinTag() { + return binTag; + } } 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..e156b419d 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryStreamReader.java @@ -25,6 +25,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Stack; import java.util.TimeZone; import java.util.UUID; @@ -91,12 +92,18 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce } } + ClickHouseDataType dataType = column.getDataType() == ClickHouseDataType.Dynamic ? readDynamicValueDataType() : column.getDataType(); + int estimatedLen = column.getEstimatedLength(); + int precision = column.getPrecision(); + int scale = column.getScale(); + TimeZone timezone = column.getTimeZoneOrDefault(timeZone); + try { - switch (column.getDataType()) { + switch (dataType) { // Primitives case FixedString: { - byte[] bytes = readNBytes(input, column.getEstimatedLength()); - return (T) new String(bytes, 0, column.getEstimatedLength(), StandardCharsets.UTF_8); + byte[] bytes = readNBytes(input, estimatedLen); + return (T) new String(bytes, 0, estimatedLen, StandardCharsets.UTF_8); } case String: { int len = readVarInt(input); @@ -132,13 +139,13 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce case Decimal: return (T) readDecimal(column.getPrecision(), column.getScale()); case Decimal32: - return (T) readDecimal(ClickHouseDataType.Decimal32.getMaxPrecision(), column.getScale()); + return (T) readDecimal(ClickHouseDataType.Decimal32.getMaxPrecision(), scale); case Decimal64: - return (T) readDecimal(ClickHouseDataType.Decimal64.getMaxPrecision(), column.getScale()); + return (T) readDecimal(ClickHouseDataType.Decimal64.getMaxPrecision(), scale); case Decimal128: - return (T) readDecimal(ClickHouseDataType.Decimal128.getMaxPrecision(), column.getScale()); + return (T) readDecimal(ClickHouseDataType.Decimal128.getMaxPrecision(), scale); case Decimal256: - return (T) readDecimal(ClickHouseDataType.Decimal256.getMaxPrecision(), column.getScale()); + return (T) readDecimal(ClickHouseDataType.Decimal256.getMaxPrecision(), scale); case Float32: return (T) Float.valueOf(readFloatLE()); case Float64: @@ -150,21 +157,15 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce case Enum16: return (T) Short.valueOf((short) readUnsignedShortLE()); case Date: - return convertDateTime(readDate(column.getTimeZone() == null ? timeZone : - column.getTimeZone()), typeHint); + return convertDateTime(readDate(timezone), typeHint); case Date32: - return convertDateTime(readDate32(column.getTimeZone() == null ? timeZone : - column.getTimeZone()), typeHint); + return convertDateTime(readDate32(timezone), typeHint); case DateTime: - return convertDateTime(readDateTime32(column.getTimeZone() == null ? timeZone : - column.getTimeZone()), typeHint); + return convertDateTime(readDateTime32(timezone), typeHint); case DateTime32: - return convertDateTime(readDateTime32(column.getTimeZone() == null ? timeZone : - column.getTimeZone()), typeHint); + return convertDateTime(readDateTime32(timezone), typeHint); case DateTime64: - return convertDateTime(readDateTime64(column.getScale(), column.getTimeZone() == null ? timeZone : - column.getTimeZone()), typeHint); - + return convertDateTime(readDateTime64(scale, timezone), typeHint); case IntervalYear: case IntervalQuarter: case IntervalMonth: @@ -214,6 +215,8 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce return (T) readValue(column.getNestedColumns().get(0)); case AggregateFunction: return (T) readBitmap( column); + case Dynamic: + return (T) readValue(column, typeHint); default: throw new IllegalArgumentException("Unsupported data type: " + column.getDataType()); } @@ -513,6 +516,35 @@ public static byte[] readNBytesLE(InputStream input, byte[] buffer, int offset, return bytes; } + public ArrayValue readArrayStack(int levels, ClickHouseColumn baseElementColumn) throws IOException { + + Stack arrays = new Stack<>(); + int level = levels; + ArrayValue array = null; + while (level <= levels) { + if (level != 1 && arrays.size() < level) { + int len = readVarInt(input); + arrays.push(new ArrayValue(ArrayValue.class, len)); + level--; + } else if (level == 1) { + int len = readVarInt(input); + array = readArrayItem(baseElementColumn, len); + level++; + } else if (array !=null) { // some array read completely + ArrayValue tmp = arrays.pop(); + if (tmp.append(array)) { // array filled + array = tmp; + level++; + } else { + array = null; + level--; + } + } + } + + return array; + } + /** * Reads a array into an ArrayValue object. * @param column - column information @@ -585,6 +617,8 @@ public static class ArrayValue { final Object array; + int nextPos = 0; + ArrayValue(Class itemType, int length) { this.itemType = itemType; this.length = length; @@ -617,6 +651,11 @@ public void set(int index, Object value) { } } + public boolean append(Object value) { + set(nextPos++, value); + return nextPos == length; + } + private List list = null; public synchronized List asList() { @@ -946,4 +985,24 @@ public byte[] allocate(int size) { return new byte[size]; } } + + private ClickHouseDataType readDynamicValueDataType() throws IOException { + byte tag = readByte(); + + ClickHouseDataType type; + if (tag == ClickHouseDataType.INTERVAL_BIN_TAG) { + byte intervalKind = readByte(); + type = ClickHouseDataType.intervalKind2Type.get(intervalKind); + if (type == null) { + throw new ClientException("Unsupported interval kind: " + intervalKind); + } + } else { + type = ClickHouseDataType.binTag2Type.get(tag); + if (type == null) { + throw new ClientException("Unsupported data type with tag " + tag); + } + } + + return type; + } } diff --git a/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java b/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java index a7fb4e43e..e3e1b05a3 100644 --- a/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java @@ -47,6 +47,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.TimeZone; import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.zip.GZIPOutputStream; 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 2e318b75c..acbb284b3 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 @@ -6,7 +6,6 @@ import com.clickhouse.client.ClickHouseNode; import com.clickhouse.client.ClickHouseProtocol; import com.clickhouse.client.ClickHouseServerForTest; -import com.clickhouse.client.ClientIntegrationTest; import com.clickhouse.client.api.Client; import com.clickhouse.client.api.ClientException; import com.clickhouse.client.api.DataTypeUtils; @@ -81,7 +80,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; @@ -2056,4 +2054,21 @@ public void testGettingRowsBeforeLimit() throws Exception { Assert.assertEquals(response.getTotalRowsToRead(), expectedTotalRowsToRead); } } + + @Test(groups = {"integration"}) + public void testGetDynamicValue() throws Exception { + String table = "test_get_dynamic_values"; + client.execute("DROP TABLE IF EXISTS " + table); + client.execute("CREATE TABLE " + table + " (rowId Int32, v Dynamic) Engine MergeTree ORDER BY ()", (CommandSettings) new CommandSettings().serverSetting("enable_dynamic_type", "1")); + + client.execute("INSERT INTO " + table + " VALUES (0, 'string'), (1, 2222222)"); + + try (QueryResponse response = client.query("SELECT * FROM " + table).get()) { + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response); + reader.next(); + System.out.println(reader.getString("v")); + reader.next(); + System.out.println(reader.getString("v")); + } + } } From 96102c989dd5e4bdf4f30886ae76dd2244102891 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 4 Feb 2025 14:16:24 -0800 Subject: [PATCH 02/21] implemented reading in rowbinary format for json and dynamic --- .../internal/BinaryStreamReader.java | 24 ++++++- .../clickhouse/client/query/QueryTests.java | 63 ++++++++++++++++--- 2 files changed, 76 insertions(+), 11 deletions(-) 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 e156b419d..b81f918c5 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 @@ -22,6 +22,7 @@ import java.time.ZonedDateTime; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -92,7 +93,7 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce } } - ClickHouseDataType dataType = column.getDataType() == ClickHouseDataType.Dynamic ? readDynamicValueDataType() : column.getDataType(); + ClickHouseDataType dataType = column.getDataType() == ClickHouseDataType.Dynamic ? readDynamicData() : column.getDataType(); int estimatedLen = column.getEstimatedLength(); int precision = column.getPrecision(); int scale = column.getScale(); @@ -199,7 +200,7 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce if (jsonAsString) { return (T) readString(input); } else { - throw new RuntimeException("Reading JSON from binary is not implemented yet"); + return (T) readJsonData(input); } // case Object: // deprecated https://clickhouse.com/docs/en/sql-reference/data-types/object-data-type case Array: @@ -986,7 +987,7 @@ public byte[] allocate(int size) { } } - private ClickHouseDataType readDynamicValueDataType() throws IOException { + private ClickHouseDataType readDynamicData() throws IOException { byte tag = readByte(); ClickHouseDataType type; @@ -1005,4 +1006,21 @@ private ClickHouseDataType readDynamicValueDataType() throws IOException { return type; } + + private static final ClickHouseColumn JSON_PLACEHOLDER_COL = ClickHouseColumn.parse("v Dynamic").get(0); + + private Map readJsonData(InputStream input) throws IOException { + int numOfPaths = readVarInt(input); + if (numOfPaths == 0) { + return Collections.emptyMap(); + } + + Map obj = new HashMap<>(); + for (int i = 0; i < numOfPaths; i++) { + String path = readString(input); + Object value = readValue(JSON_PLACEHOLDER_COL); + obj.put(path, value); + } + return obj; + } } 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 acbb284b3..40523c19a 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 @@ -80,9 +80,11 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; +import java.util.regex.Matcher; import java.util.stream.BaseStream; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -1420,7 +1422,10 @@ private List> prepareDataSet(String table, List colu } createStmtBuilder.setLength(createStmtBuilder.length() - 2); createStmtBuilder.append(") ENGINE = MergeTree ORDER BY tuple()"); - client.execute(createStmtBuilder.toString()).get(10, TimeUnit.SECONDS); + client.execute(createStmtBuilder.toString(), (CommandSettings) + new CommandSettings().serverSetting("enable_dynamic_type", "1") + .serverSetting("allow_experimental_json_type", "1")) + .get(10, TimeUnit.SECONDS); // Insert data StringBuilder insertStmtBuilder = new StringBuilder(); @@ -2058,17 +2063,59 @@ public void testGettingRowsBeforeLimit() throws Exception { @Test(groups = {"integration"}) public void testGetDynamicValue() throws Exception { String table = "test_get_dynamic_values"; - client.execute("DROP TABLE IF EXISTS " + table); - client.execute("CREATE TABLE " + table + " (rowId Int32, v Dynamic) Engine MergeTree ORDER BY ()", (CommandSettings) new CommandSettings().serverSetting("enable_dynamic_type", "1")); - client.execute("INSERT INTO " + table + " VALUES (0, 'string'), (1, 2222222)"); + final AtomicInteger rowId = new AtomicInteger(-1); + final Random rnd = new Random(); + + List> dataset = prepareDataSet(table, Arrays.asList("rowId Int32", "v Dynamic"), + Arrays.asList(s -> rowId.incrementAndGet(), s-> { + int decision = rnd.nextInt(3); + if (decision == 0) { + return RandomStringUtils.randomAlphanumeric(3, 10); + } else if (decision == 1) { + return rnd.nextInt(); + } else { + return rnd.nextDouble(); + } + }), 1000); try (QueryResponse response = client.query("SELECT * FROM " + table).get()) { ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response); - reader.next(); - System.out.println(reader.getString("v")); - reader.next(); - System.out.println(reader.getString("v")); + while (reader.next() != null) { + int rowIndex = reader.getInteger("rowId"); + Assert.assertEquals(reader.getString("v"), dataset.get(rowIndex).get("v").toString()); + } + } + } + + @Test(groups = {"integration"}) + public void testGetJSON() throws Exception { + String table = "test_get_json_values"; + + final AtomicInteger rowId = new AtomicInteger(-1); + final Random rnd = new Random(); + + List> dataset = prepareDataSet(table, Arrays.asList("rowId Int32", "v1 JSON"), + Arrays.asList(s -> rowId.incrementAndGet(), + s-> { + String a = "{'a': '" + RandomStringUtils.randomAlphabetic(20) + "', 'b': { 'c': 'test1', 'd': " + rnd + .nextInt(1000) + "}}"; + return a.replaceAll("'", "\""); + }), 1); + + System.out.println(dataset); + ObjectMapper jackson = new ObjectMapper(); + try (QueryResponse response = client.query("SELECT * FROM " + table).get()) { + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response); + while (reader.next() != null) { + int rowIndex = reader.getInteger("rowId"); + JsonNode expected = jackson.readValue(dataset.get(rowIndex).get("v1").toString(), JsonNode.class); + Map v1 = reader.readValue("v1"); + for (Map.Entry e : v1.entrySet()) { + String pointer = "/" + e.getKey().replaceAll("\\.", "/"); + Assert.assertEquals(e.getValue().toString(), expected.at(pointer).asText()); + } + } } } } From 4f8de9df5e409abd41bab03c039649e77f61cbf4 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 5 Feb 2025 12:54:17 -0800 Subject: [PATCH 03/21] fixed tests for ch versions --- .../client/datatypes/DataTypeTests.java | 13 +++++++ .../clickhouse/client/query/QueryTests.java | 38 +++++++++++-------- 2 files changed, 36 insertions(+), 15 deletions(-) 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 bbb5fa02d..c6f8db527 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 @@ -12,6 +12,7 @@ import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.GenericRecord; import com.clickhouse.data.ClickHouseDataType; +import com.clickhouse.data.ClickHouseVersion; import lombok.AllArgsConstructor; import lombok.Data; import lombok.NoArgsConstructor; @@ -142,6 +143,10 @@ public static String tblCreateSQL(String table) { @Test(groups = {"integration"}) public void testVariantWithSimpleDataTypes() throws Exception { + if (isVersionMatch("(,24.8]")) { + return; + } + final String table = "test_variant_primitives"; final DataTypesTestingPOJO sample = new DataTypesTestingPOJO(); @@ -383,6 +388,10 @@ public void testVariantWithTuple() throws Exception { } private void testVariantWith(String withWhat, String[] fields, Object[] values, String[] expectedStrValues) throws Exception { + if (isVersionMatch("(,24.8]")) { + return; + } + String table = "test_variant_with_" + withWhat; String[] actualFields = new String[fields.length + 1]; actualFields[0] = "rowId Int32"; @@ -416,4 +425,8 @@ public static String tableDefinition(String table, String... columns) { return sb.toString(); } + public boolean isVersionMatch(String versionExpression) { + List serverVersion = client.queryAll("SELECT version()"); + return ClickHouseVersion.of(serverVersion.get(0).getString(1)).check(versionExpression); + } } 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 d0d6a93ae..52b618603 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 @@ -1418,6 +1418,11 @@ private List> prepareDataSet(String table, List colu client.execute("DROP TABLE IF EXISTS " + table).get(10, TimeUnit.SECONDS); // Create table + CommandSettings settings = new CommandSettings(); + if (isVersionMatch("[24.8,)")) { + settings.serverSetting("enable_dynamic_type", "1") + .serverSetting("allow_experimental_json_type", "1"); + } StringBuilder createStmtBuilder = new StringBuilder(); createStmtBuilder.append("CREATE TABLE IF NOT EXISTS ").append(table).append(" ("); for (String column : columns) { @@ -1425,10 +1430,7 @@ private List> prepareDataSet(String table, List colu } createStmtBuilder.setLength(createStmtBuilder.length() - 2); createStmtBuilder.append(") ENGINE = MergeTree ORDER BY tuple()"); - client.execute(createStmtBuilder.toString(), (CommandSettings) - new CommandSettings().serverSetting("enable_dynamic_type", "1") - .serverSetting("allow_experimental_json_type", "1")) - .get(10, TimeUnit.SECONDS); + client.execute(createStmtBuilder.toString(), settings).get(10, TimeUnit.SECONDS); // Insert data StringBuilder insertStmtBuilder = new StringBuilder(); @@ -1787,9 +1789,7 @@ private static Object[][] sessionRoles() { @Test(groups = {"integration"}, dataProvider = "sessionRoles", dataProviderClass = QueryTests.class) public void testOperationCustomRoles(String[] roles) throws Exception { - List serverVersion = client.queryAll("SELECT version()"); - if (ClickHouseVersion.of(serverVersion.get(0).getString(1)).check("(,24.3]")) { - System.out.println("Test is skipped: feature is supported since 24.4"); + if (isVersionMatch("(,24.3]")) { return; } @@ -1825,9 +1825,7 @@ private static Object[][] clientSessionRoles() { } @Test(groups = {"integration"}, dataProvider = "clientSessionRoles", dataProviderClass = QueryTests.class) public void testClientCustomRoles(String[] roles) throws Exception { - List serverVersion = client.queryAll("SELECT version()"); - if (ClickHouseVersion.of(serverVersion.get(0).getString(1)).check("(,24.3]")) { - System.out.println("Test is skipped: feature is supported since 24.4"); + if (isVersionMatch("(,24.3]")) { return; } @@ -1909,9 +1907,7 @@ public void testReadingJSONValues() throws Exception { if (isCloud()) { return; // TODO: add support on cloud } - List serverVersion = client.queryAll("SELECT version()"); - if (ClickHouseVersion.of(serverVersion.get(0).getString(1)).check("(,24.8]")) { - System.out.println("Test is skipped: feature is supported since 24.8"); + if (isVersionMatch("(,24.8]")) { return; } CommandSettings commandSettings = new CommandSettings(); @@ -2050,8 +2046,7 @@ public void testLowCardinalityValues() throws Exception { @Test(groups = {"integration"}) public void testGettingRowsBeforeLimit() throws Exception { int expectedTotalRowsToRead = 100; - List serverVersion = client.queryAll("SELECT version()"); - if (ClickHouseVersion.of(serverVersion.get(0).getString(1)).check("(,23.8]")) { + if (isVersionMatch("(,23.8]")) { // issue in prev. release. expectedTotalRowsToRead = 0; } @@ -2065,6 +2060,10 @@ public void testGettingRowsBeforeLimit() throws Exception { @Test(groups = {"integration"}) public void testGetDynamicValue() throws Exception { + if (isVersionMatch("(,24.8]")) { + return; + } + String table = "test_get_dynamic_values"; final AtomicInteger rowId = new AtomicInteger(-1); @@ -2093,6 +2092,10 @@ public void testGetDynamicValue() throws Exception { @Test(groups = {"integration"}) public void testGetJSON() throws Exception { + if (isVersionMatch("(,24.8]")) { + return; + } + String table = "test_get_json_values"; final AtomicInteger rowId = new AtomicInteger(-1); @@ -2121,4 +2124,9 @@ public void testGetJSON() throws Exception { } } } + + public boolean isVersionMatch(String versionExpression) { + List serverVersion = client.queryAll("SELECT version()"); + return ClickHouseVersion.of(serverVersion.get(0).getString(1)).check(versionExpression); + } } From 2ae4d27de03f928bacf9c8211bd85ff5f72e4f31 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Fri, 7 Feb 2025 16:44:27 -0800 Subject: [PATCH 04/21] implemented dynamic basic types --- .../client/ClickHouseServerForTest.java | 6 +- .../clickhouse-server/users.d/users.xml | 9 + .../clickhouse/data/ClickHouseDataType.java | 7 + .../internal/BinaryStreamReader.java | 4 + .../internal/SerializerUtils.java | 213 +++++++++++++++++- .../client/datatypes/DataTypeTests.java | 99 +++++++- 6 files changed, 331 insertions(+), 7 deletions(-) diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseServerForTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseServerForTest.java index 3c3fdcaab..f2d1ae055 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseServerForTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseServerForTest.java @@ -250,7 +250,9 @@ public static ClickHouseNode getClickHouseNode(ClickHouseProtocol protocol, bool } } - return ClickHouseNode.builder(template).address(protocol, new InetSocketAddress(host, port)).build(); + return ClickHouseNode.builder(template).address(protocol, new InetSocketAddress(host, port)) + .credentials(new ClickHouseCredentials("default", getPassword())) + .build(); } public static ClickHouseNode getClickHouseNode(ClickHouseProtocol protocol, int port) { @@ -314,7 +316,7 @@ public static String getPassword() { if (isCloud) { return System.getenv("CLICKHOUSE_CLOUD_PASSWORD"); } else { - return ""; + return "test_default_password"; } } diff --git a/clickhouse-client/src/test/resources/containers/clickhouse-server/users.d/users.xml b/clickhouse-client/src/test/resources/containers/clickhouse-server/users.d/users.xml index efe565b20..b020f502a 100644 --- a/clickhouse-client/src/test/resources/containers/clickhouse-server/users.d/users.xml +++ b/clickhouse-client/src/test/resources/containers/clickhouse-server/users.d/users.xml @@ -10,6 +10,15 @@ + + 0 + default + + ::/0 + + test_default_password + default + 1 default 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 ba4ec582c..af51473d9 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -228,6 +228,8 @@ private static Set> setOf(Class... args) { public static final byte SET_BIN_TAG = 0x21; + public static final byte CUSTOM_TYPE_BIN_TAG = 0x2C; + public enum IntervalKindBinTag { Nanosecond(IntervalNanosecond, 0x00), Microsecond(IntervalMicrosecond, 0x01), @@ -284,6 +286,8 @@ public byte getTag() { public static final Map intervalKind2Type; + public static final Map intervalType2Kind; + static { Set set = new TreeSet<>(); Map map = new HashMap<>(); @@ -320,10 +324,13 @@ public byte getTag() { binTag2Type = Collections.unmodifiableMap(tmpbinTag2Type); Map tmpIntervalKind2Type = new HashMap<>(); + Map tmpIntervalType2Kind = new HashMap<>(); for (IntervalKindBinTag kind : IntervalKindBinTag.values()) { tmpIntervalKind2Type.put(kind.getTag(), kind.getIntervalType()); + tmpIntervalType2Kind.put(kind.getIntervalType(), kind.tag); } intervalKind2Type = Collections.unmodifiableMap(tmpIntervalKind2Type); + intervalType2Kind = Collections.unmodifiableMap(tmpIntervalType2Kind); } /** 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 7afab0dd8..93e2d8167 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 @@ -1007,7 +1007,11 @@ private ClickHouseDataType readDynamicData() throws IOException { if (type == null) { throw new ClientException("Unsupported interval kind: " + intervalKind); } + } else { + if (tag == ClickHouseDataType.String.getBinTag()) { + System.out.println("String"); + } type = ClickHouseDataType.binTag2Type.get(tag); if (type == null) { throw new ClientException("Unsupported data type with tag " + tag); 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 3e6b6ba71..eb13f6428 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,8 +2,6 @@ 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; @@ -30,17 +28,21 @@ import java.net.Inet4Address; import java.net.Inet6Address; import java.sql.Timestamp; +import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.ZoneId; import java.time.ZonedDateTime; import java.util.Arrays; +import java.util.Collections; +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.stream.Collectors; @@ -90,6 +92,11 @@ public static void serializeData(OutputStream stream, Object value, ClickHouseCo value = value instanceof ClickHouseGeoMultiPolygonValue ? ((ClickHouseGeoMultiPolygonValue)value).getValue() : value; serializeArrayData(stream, value, GEO_MULTI_POLYGON_ARRAY); break; + case Dynamic: + ClickHouseColumn typeColumn = valueToColumnForDynamicType(value); + writeDynamicTypeTag(stream, typeColumn, value); + serializeData(stream, value, typeColumn); + break; default: serializePrimitiveData(stream, value, column); break; @@ -97,7 +104,207 @@ public static void serializeData(OutputStream stream, Object value, ClickHouseCo } } - private static void serializeArrayData(OutputStream stream, Object value, ClickHouseColumn column) throws IOException { + private static final Map, ClickHouseColumn> PREDEFINED_TYPE_COLUMNS = getPredefinedTypeColumnsMap(); + + private static Map, ClickHouseColumn> getPredefinedTypeColumnsMap() { + HashMap, ClickHouseColumn> map = new HashMap<>(); + map.put(Void.class, ClickHouseColumn.of("v", "Nothing")); + map.put(Boolean.class, ClickHouseColumn.of("v", "Bool")); + map.put(Byte.class, ClickHouseColumn.of("v", "Int8")); + map.put(Short.class, ClickHouseColumn.of("v", "Int16")); + map.put(Integer.class, ClickHouseColumn.of("v", "Int32")); + map.put(Long.class, ClickHouseColumn.of("v", "Int64")); + map.put(BigInteger.class, ClickHouseColumn.of("v", "Int256")); + map.put(Float.class, ClickHouseColumn.of("v", "Float32")); + map.put(Double.class, ClickHouseColumn.of("v", "Float64")); + map.put(LocalDate.class, ClickHouseColumn.of("v", "Date")); + map.put(UUID.class, ClickHouseColumn.of("v", "UUID")); + map.put(Inet4Address.class, ClickHouseColumn.of("v", "IPv4")); + map.put(Inet6Address.class, ClickHouseColumn.of("v", "IPv6")); + map.put(String.class, ClickHouseColumn.of("v", "String")); + map.put(LocalDateTime.class, ClickHouseColumn.of("v", "DateTime")); + + map.put(boolean[].class, ClickHouseColumn.of("v", "Array(Bool)")); + map.put(boolean[][].class, ClickHouseColumn.of("v", "Array(Array(Bool))")); + map.put(boolean[][][].class, ClickHouseColumn.of("v", "Array(Array(Array(Bool)))")); + + map.put(byte[].class, ClickHouseColumn.of("v", "Array(Int8)")); + map.put(byte[][].class, ClickHouseColumn.of("v", "Array(Array(Int8))")); + map.put(byte[][][].class, ClickHouseColumn.of("v", "Array(Array(Array(Int8)))")); + + map.put(short[].class, ClickHouseColumn.of("v", "Array(Int16)")); + map.put(short[][].class, ClickHouseColumn.of("v", "Array(Array(Int16))")); + map.put(short[][][].class, ClickHouseColumn.of("v", "Array(Array(Array(Int16)))")); + + map.put(int[].class, ClickHouseColumn.of("v", "Array(Int32)")); + map.put(int[][].class, ClickHouseColumn.of("v", "Array(Array(Int32))")); + map.put(int[][][].class, ClickHouseColumn.of("v", "Array(Array(Array(Int32)))")); + + map.put(long[].class, ClickHouseColumn.of("v", "Array(Int64)")); + map.put(long[][].class, ClickHouseColumn.of("v", "Array(Array(Int64))")); + map.put(long[][][].class, ClickHouseColumn.of("v", "Array(Array(Array(Int64)))")); + + map.put(float[].class, ClickHouseColumn.of("v", "Array(Float32)")); + map.put(float[][].class, ClickHouseColumn.of("v", "Array(Array(Float32))")); + map.put(float[][][].class, ClickHouseColumn.of("v", "Array(Array(Array(Float32)))")); + + map.put(double[].class, ClickHouseColumn.of("v", "Array(Float64)")); + map.put(double[][].class, ClickHouseColumn.of("v", "Array(Array(Float64))")); + map.put(double[][][].class, ClickHouseColumn.of("v", "Array(Array(Array(Float64)))")); + + return Collections.unmodifiableMap(map); + } + + public static ClickHouseColumn valueToColumnForDynamicType(Object value) { + ClickHouseColumn column; + if (value instanceof ZonedDateTime) { + ZonedDateTime dt = (ZonedDateTime) value; + column = ClickHouseColumn.of("v", "DateTime(" + dt.getZone().getId() + ")"); + } else if (value instanceof BigDecimal) { + BigDecimal d = (BigDecimal) value; + column = ClickHouseColumn.of("v", "Decimal256(" + d.precision() + ", " + d.scale() + ")"); + } else if (value instanceof Map) { + Map map = (Map) value; + // TODO: handle empty map? + Map.Entry entry = map.entrySet().iterator().next(); + ClickHouseColumn keyInfo = valueToColumnForDynamicType(entry.getKey()); + ClickHouseColumn valueInfo = valueToColumnForDynamicType(entry.getValue()); + column = ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + ", " + valueInfo.getOriginalTypeName() + ")"); + } else if (value instanceof List) { + List list = (List) value; + StringBuilder type = new StringBuilder("Array()"); + int insertPos = type.length() - 2; + while (!list.isEmpty() && list.get(0) instanceof List) { + type.insert(insertPos, "Array()"); + insertPos += 6; // add len of 'Array(' string + list = (List) list.get(0); + } + if (list.isEmpty()) { + type.insert(insertPos, "Nothing"); + column = ClickHouseColumn.of("v", type.toString()); + } else { + ClickHouseColumn arrayBaseColumn = PREDEFINED_TYPE_COLUMNS.get(list.get(0)); + if (arrayBaseColumn != null) { + type.insert(insertPos, arrayBaseColumn.getOriginalTypeName()); + column = ClickHouseColumn.of("v", type.toString()); + } else { + column = null; + } + } + } else if (value == null) { + column = PREDEFINED_TYPE_COLUMNS.get(Void.class); + } else { + column = PREDEFINED_TYPE_COLUMNS.get(value.getClass()); + } + + if (column == null) { + throw new ClientException("Unable to serialize value of " + value.getClass() + " because not supported yet"); + } + + return column; + } + + public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typeColumn, Object value) + throws IOException { + + ClickHouseDataType dt = typeColumn.getDataType(); + byte binTag = dt.getBinTag(); + if (binTag == -1) { + throw new ClientException("Type " + dt.name() +" serialization is not supported for Dynamic column"); + } + + if (typeColumn.isNullable()) { + stream.write(ClickHouseDataType.NULLABLE_BIN_TAG); + } + if (typeColumn.isLowCardinality()) { + stream.write(ClickHouseDataType.LOW_CARDINALITY_BIN_TAG); + } + + switch (dt) { + case FixedString: + stream.write(binTag); + writeVarInt(stream, typeColumn.getEstimatedLength()); + break; + case Enum8: + stream.write(binTag); + /// 0x17... + break; + case Enum16: + stream.write(binTag); + //0x18...> + break; + case Decimal: + case Decimal32: + case Decimal64: + case Decimal128: + case Decimal256: + stream.write(binTag); + // + break; + + case IntervalNanosecond: + case IntervalMillisecond: + case IntervalSecond: + case IntervalMinute: + case IntervalHour: + case IntervalDay: + case IntervalWeek: + case IntervalMonth: + case IntervalQuarter: + case IntervalYear: + stream.write(binTag); + stream.write(ClickHouseDataType.IntervalKindBinTag.Day.getTag()); + break; + case DateTime32: + stream.write(binTag); + BinaryStreamUtils.writeString(stream, typeColumn.getTimeZoneOrDefault(TimeZone.getDefault()).getID()); + break; + case DateTime64: + break; + case Array: + stream.write(binTag); + // elements 0x1E + break; + case Map: + stream.write(binTag); + ///0x0F... + break; + case Tuple: + // Tuple(T1, ..., TN) + // 0x1F... + stream.write(0x1F); + // or + // Tuple(name1 T1, ..., nameN TN) + // 0x20... + stream.write(0x20); + break; + case Point: + case Polygon: + case Ring: + case MultiPolygon: + stream.write(ClickHouseDataType.CUSTOM_TYPE_BIN_TAG); + break; + case Variant: + stream.write(binTag); + break; + case Dynamic: + stream.write(binTag); + break; + case JSON: + stream.write(binTag); + break; + case SimpleAggregateFunction: + stream.write(binTag); + break; + case AggregateFunction: + stream.write(binTag); + break; + default: + stream.write(binTag); + } + } + + public static void serializeArrayData(OutputStream stream, Object value, ClickHouseColumn column) throws IOException { if (value instanceof List) { //Serialize the array to the stream 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 c6f8db527..5887dfdc9 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 @@ -61,7 +61,7 @@ public void setUp() throws IOException { client = new Client.Builder() .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), false) .setUsername("default") - .setPassword("") + .setPassword(getPassword()) .useNewImplementation(System.getProperty("client.tests.useNewImplementation", "true").equals("true")) .compressClientRequest(useClientCompression) .useHttpCompression(useHttpCompression) @@ -146,7 +146,7 @@ public void testVariantWithSimpleDataTypes() throws Exception { if (isVersionMatch("(,24.8]")) { return; } - + final String table = "test_variant_primitives"; final DataTypesTestingPOJO sample = new DataTypesTestingPOJO(); @@ -387,6 +387,101 @@ public void testVariantWithTuple() throws Exception { }); } + + + @Test(groups = {"integration"}) + public void testDynamicWithPrimitives() throws Exception { + + if (isVersionMatch("(,24.8]")) { + return; + } + + final String table = "test_dynamic_primitives"; + final DataTypesTestingPOJO sample = new DataTypesTestingPOJO(); + + client.execute("DROP TABLE IF EXISTS " + table).get(); + String createTableStatement = " CREATE TABLE " + table + "( rowId Int64, field Dynamic ) " + + "Engine = MergeTree ORDER BY ()"; + + client.execute(createTableStatement, (CommandSettings) new CommandSettings().serverSetting("enable_dynamic_type", "1")); + client.register(DTOForDynamicPrimitivesTests.class, client.getTableSchema(table)); + + int rowId = 0; + for (ClickHouseDataType dataType : ClickHouseDataType.values()) { + System.out.println("Testing dynamic with " + dataType + " values"); + + switch (dataType) { + case Date: + case Date32: + case DateTime: + case DateTime32: + case DateTime64: + case Enum8: + case Enum16: + continue; + default: + } + + 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 DTOForDynamicPrimitivesTests(rowId++, value)); + try { + client.insert(table, data).get().close(); + } catch (Exception e) { + System.out.println("Failed for " + dataType + ": " + e.getMessage()); + continue; + } + List rows = client.queryAll("SELECT * FROM " + table + " ORDER BY rowId DESC "); + GenericRecord row = rows.get(0); + 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)); + } + } + } + + @Data + @AllArgsConstructor + public static class DTOForDynamicPrimitivesTests { + private int rowId; + private Object field; + } + private void testVariantWith(String withWhat, String[] fields, Object[] values, String[] expectedStrValues) throws Exception { if (isVersionMatch("(,24.8]")) { return; From af28fcab4274ff0a7b77c8a50efa603049657264 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Sat, 8 Feb 2025 00:25:09 -0800 Subject: [PATCH 05/21] implemented interval and partially decimal --- .../clickhouse/data/ClickHouseDataType.java | 4 ++ .../internal/AbstractBinaryFormatReader.java | 1 + .../internal/BinaryStreamReader.java | 34 +++++++++++-- .../internal/SerializerUtils.java | 27 +++++++++- .../datatypes/DataTypesTestingPOJO.java | 50 ++++++++++++++++++- 5 files changed, 109 insertions(+), 7 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 af51473d9..a3b1ca773 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -230,6 +230,10 @@ private static Set> setOf(Class... args) { public static final byte CUSTOM_TYPE_BIN_TAG = 0x2C; + public static final byte TUPLE_WITHOUT_NAMES_BIN_TAG = 0x1F; + + public static final byte TUPLE_WITH_NAMES_BIN_TAG = 0x20; + public enum IntervalKindBinTag { Nanosecond(IntervalNanosecond, 0x00), Microsecond(IntervalMicrosecond, 0x01), 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 cfbe233cc..b623c6806 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 @@ -252,6 +252,7 @@ protected void setSchema(TableSchema schema) { case Enum8: case Enum16: case Variant: + case Dynamic: 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 93e2d8167..91799b999 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 @@ -1005,13 +1005,37 @@ private ClickHouseDataType readDynamicData() throws IOException { byte intervalKind = readByte(); type = ClickHouseDataType.intervalKind2Type.get(intervalKind); if (type == null) { - throw new ClientException("Unsupported interval kind: " + intervalKind); + throw new ClientException("Unsupported interval kind: " + intervalKind); } - + } else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) { + String typeName = readString(input); + return ClickHouseDataType.valueOf(typeName); + } else if (tag == ClickHouseDataType.Decimal.getBinTag()) { + byte precision = readByte(); + byte scale = readByte(); + System.out.println("precision: " + precision + " scale: " + scale); + return ClickHouseDataType.Decimal; + } else if (tag == ClickHouseDataType.Decimal32.getBinTag()) { + byte precision = readByte(); + byte scale = readByte(); + System.out.println("precision: " + precision + " scale: " + scale); + return ClickHouseDataType.Decimal; + } else if (tag == ClickHouseDataType.Decimal64.getBinTag()) { + byte precision = readByte(); + byte scale = readByte(); + System.out.println("precision: " + precision + " scale: " + scale); + return ClickHouseDataType.Decimal64; + } else if (tag == ClickHouseDataType.Decimal128.getBinTag()) { + byte precision = readByte(); + byte scale = readByte(); + System.out.println("precision: " + precision + " scale: " + scale); + return ClickHouseDataType.Decimal128; + } else if (tag == ClickHouseDataType.Decimal256.getBinTag()) { + byte precision = readByte(); + byte scale = readByte(); + System.out.println("precision: " + precision + " scale: " + scale); + return ClickHouseDataType.Decimal256; } else { - if (tag == ClickHouseDataType.String.getBinTag()) { - System.out.println("String"); - } type = ClickHouseDataType.binTag2Type.get(tag); if (type == null) { throw new ClientException("Unsupported data type with tag " + tag); 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 eb13f6428..5e537289f 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 @@ -33,6 +33,7 @@ import java.time.LocalDateTime; import java.time.ZoneId; import java.time.ZonedDateTime; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -124,6 +125,11 @@ private static Map, ClickHouseColumn> getPredefinedTypeColumnsMap() { map.put(String.class, ClickHouseColumn.of("v", "String")); map.put(LocalDateTime.class, ClickHouseColumn.of("v", "DateTime")); + map.put(ClickHouseGeoPointValue.class, ClickHouseColumn.of("v", "Point")); + map.put(ClickHouseGeoRingValue.class, ClickHouseColumn.of("v", "Ring")); + map.put(ClickHouseGeoPolygonValue.class, ClickHouseColumn.of("v", "Polygon")); + map.put(ClickHouseGeoMultiPolygonValue.class, ClickHouseColumn.of("v", "MultiPolygon")); + map.put(boolean[].class, ClickHouseColumn.of("v", "Array(Bool)")); map.put(boolean[][].class, ClickHouseColumn.of("v", "Array(Array(Bool))")); map.put(boolean[][][].class, ClickHouseColumn.of("v", "Array(Array(Array(Bool)))")); @@ -239,6 +245,8 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ case Decimal128: case Decimal256: stream.write(binTag); + stream.write(dt.getMaxPrecision()); + stream.write(dt.getMaxScale()); // break; @@ -253,7 +261,11 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ case IntervalQuarter: case IntervalYear: stream.write(binTag); - stream.write(ClickHouseDataType.IntervalKindBinTag.Day.getTag()); + Byte kindTag = ClickHouseDataType.intervalType2Kind.get(dt); + if (kindTag == null) { + throw new ClientException("BUG! No Interval Mapping to a kind tag"); + } + stream.write(kindTag); break; case DateTime32: stream.write(binTag); @@ -283,6 +295,7 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ case Ring: case MultiPolygon: stream.write(ClickHouseDataType.CUSTOM_TYPE_BIN_TAG); + BinaryStreamUtils.writeString(stream, dt.name()); break; case Variant: stream.write(binTag); @@ -469,6 +482,18 @@ private static void serializePrimitiveData(OutputStream stream, Object value, Cl case JSON: serializeJSON(stream, value); break; + case IntervalNanosecond: + case IntervalMillisecond: + case IntervalSecond: + case IntervalMinute: + case IntervalHour: + case IntervalDay: + case IntervalWeek: + case IntervalMonth: + case IntervalQuarter: + case IntervalYear: + BinaryStreamUtils.writeUnsignedInt64(stream, convertToLong(value)); + break; default: throw new UnsupportedOperationException("Unsupported data type: " + column.getDataType()); } 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 index 90b6594ce..042100504 100644 --- a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java @@ -94,6 +94,28 @@ public class DataTypesTestingPOJO { private ClickHouseBitmap groupBitmapUint32; private ClickHouseBitmap groupBitmapUint64; + private int intervalYear; + + private byte intervalQuarter; + + private byte intervalMonth; + + private byte intervalWeek; + + private short intervalDay; + + private byte intervalHour; + + private byte intervalMinute; + + private byte intervalSecond; + + private long intervalMillisecond; + + private long intervalMicrosecond; + + private BigInteger intervalNanosecond; + public DataTypesTestingPOJO() { final Random random = new Random(); byteValue = (byte) random.nextInt(); @@ -219,6 +241,21 @@ public DataTypesTestingPOJO() { 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()); + intervalYear = random.nextInt(2000, 4000); + intervalQuarter = (byte) random.nextInt(4); + intervalMonth = (byte) random.nextInt(12); + intervalWeek = (byte) random.nextInt(52); + intervalDay = (byte) random.nextInt(30); + intervalHour = (byte) random.nextInt(24); + intervalMinute = (byte) random.nextInt(60); + intervalSecond = (byte) random.nextInt(60); + intervalMillisecond = random.nextLong(10000); + intervalMicrosecond = random.nextLong(10000); + + upper = BigInteger.valueOf(random.nextLong()).shiftLeft(64); + lower = BigInteger.valueOf(random.nextLong()).and(BigInteger.valueOf(Long.MAX_VALUE)); + + intervalNanosecond = upper.or(lower); } public boolean getBool() { @@ -277,7 +314,18 @@ public static String generateTableCreateSQL(String tableName) { "nested Nested (innerInt Int32, innerString String, " + "innerNullableInt Nullable(Int32)), " + "groupBitmapUint32 AggregateFunction(groupBitmap, UInt32), " + - "groupBitmapUint64 AggregateFunction(groupBitmap, UInt64) " + + "groupBitmapUint64 AggregateFunction(groupBitmap, UInt64), " + + "intervalYear IntervalYear, " + + "intervalQuarter IntervalQuarter, " + + "intervalMonth IntervalMonth, " + + "intervalWeek IntervalWeek, " + + "intervalDay IntervalDay, " + + "intervalHour IntervalHour, " + + "intervalMinute IntervalMinute, " + + "intervalSecond IntervalSecond, " + + "intervalMillisecond IntervalMillisecond, " + + "intervalMicrosecond IntervalMicrosecond, " + + "intervalNanosecond IntervalNanosecond " + ") ENGINE = MergeTree ORDER BY ()"; } } From d2b6d59066e36f3c5d643ae1d2f33114d9e6e0a2 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Mon, 10 Feb 2025 08:33:00 -0800 Subject: [PATCH 06/21] almost all types are working --- .../internal/BinaryStreamReader.java | 33 +++++++------------ .../internal/SerializerUtils.java | 12 ++++++- .../client/datatypes/DataTypeTests.java | 4 +++ 3 files changed, 26 insertions(+), 23 deletions(-) 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 91799b999..54d64cefa 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 @@ -97,6 +97,13 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce int estimatedLen = column.getEstimatedLength(); int precision = column.getPrecision(); int scale = column.getScale(); + if (dataType == ClickHouseDataType.Decimal || dataType == ClickHouseDataType.Decimal32 || + dataType == ClickHouseDataType.Decimal64 || dataType == ClickHouseDataType.Decimal128 || + dataType == ClickHouseDataType.Decimal256) { + precision = readByte(); + scale = readByte(); + System.out.println("p: " + precision + " " +scale); + } TimeZone timezone = column.getTimeZoneOrDefault(timeZone); try { @@ -138,7 +145,7 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce case UInt256: return (T) readBigIntegerLE(INT256_SIZE, true); case Decimal: - return (T) readDecimal(column.getPrecision(), column.getScale()); + return (T) readDecimal(precision, scale); case Decimal32: return (T) readDecimal(ClickHouseDataType.Decimal32.getMaxPrecision(), scale); case Decimal64: @@ -462,7 +469,6 @@ public BigDecimal readDecimal(int precision, int scale) throws IOException { } else { v = new BigDecimal(readBigIntegerLE(INT256_SIZE, false), scale); } - return v; } @@ -1010,31 +1016,14 @@ private ClickHouseDataType readDynamicData() throws IOException { } else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) { String typeName = readString(input); return ClickHouseDataType.valueOf(typeName); - } else if (tag == ClickHouseDataType.Decimal.getBinTag()) { - byte precision = readByte(); - byte scale = readByte(); - System.out.println("precision: " + precision + " scale: " + scale); - return ClickHouseDataType.Decimal; } else if (tag == ClickHouseDataType.Decimal32.getBinTag()) { - byte precision = readByte(); - byte scale = readByte(); - System.out.println("precision: " + precision + " scale: " + scale); return ClickHouseDataType.Decimal; } else if (tag == ClickHouseDataType.Decimal64.getBinTag()) { - byte precision = readByte(); - byte scale = readByte(); - System.out.println("precision: " + precision + " scale: " + scale); - return ClickHouseDataType.Decimal64; + return ClickHouseDataType.Decimal; } else if (tag == ClickHouseDataType.Decimal128.getBinTag()) { - byte precision = readByte(); - byte scale = readByte(); - System.out.println("precision: " + precision + " scale: " + scale); - return ClickHouseDataType.Decimal128; + return ClickHouseDataType.Decimal; } else if (tag == ClickHouseDataType.Decimal256.getBinTag()) { - byte precision = readByte(); - byte scale = readByte(); - System.out.println("precision: " + precision + " scale: " + scale); - return ClickHouseDataType.Decimal256; + return ClickHouseDataType.Decimal; } else { type = ClickHouseDataType.binTag2Type.get(tag); if (type == null) { 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 5e537289f..07287de54 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 @@ -168,7 +168,17 @@ public static ClickHouseColumn valueToColumnForDynamicType(Object value) { column = ClickHouseColumn.of("v", "DateTime(" + dt.getZone().getId() + ")"); } else if (value instanceof BigDecimal) { BigDecimal d = (BigDecimal) value; - column = ClickHouseColumn.of("v", "Decimal256(" + d.precision() + ", " + d.scale() + ")"); + String decType; + if (d.scale() <= ClickHouseDataType.Decimal32.getMaxScale()) { + decType = "Decimal32"; + } else if (d.scale() <= ClickHouseDataType.Decimal64.getMaxScale()) { + decType = "Decimal64"; + } else if (d.scale() <= ClickHouseDataType.Decimal128.getMaxScale()) { + decType = "Decimal128"; + } else { + decType = "Decimal256"; + } + column = ClickHouseColumn.of("v", decType + "(" + d.precision() +"," + d.scale() + ")"); } else if (value instanceof Map) { Map map = (Map) value; // TODO: handle empty map? 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 2e8dc062c..f45753e68 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 @@ -466,6 +466,10 @@ public void testDynamicWithPrimitives() throws Exception { case MultiPolygon: strValue = row.getGeoMultiPolygon("field").toString(); break; + case Decimal32: + double v = row.getDouble("field"); + System.out.println(v); + break; } System.out.println("field: " + strValue + " value " + value); if (value.getClass().isPrimitive()) { From 9c3a6edd72f220510653fbc4a188762c8b533258 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Mon, 10 Feb 2025 13:08:15 -0800 Subject: [PATCH 07/21] fixed array handling when Dynamic --- .../com/clickhouse/data/ClickHouseColumn.java | 2 +- .../internal/BinaryStreamReader.java | 66 +++++++------- .../internal/SerializerUtils.java | 53 +++++++++--- .../client/datatypes/DataTypeTests.java | 85 ++++++++++++++++++- .../datatypes/DataTypesTestingPOJO.java | 6 +- 5 files changed, 162 insertions(+), 50 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 d723c65d6..d2eb55d96 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java @@ -640,7 +640,7 @@ public static List parse(String args) { return Collections.unmodifiableList(c); } - private ClickHouseColumn(ClickHouseDataType dataType, String columnName, String originalTypeName, boolean nullable, + public ClickHouseColumn(ClickHouseDataType dataType, String columnName, String originalTypeName, boolean nullable, boolean lowCardinality, List parameters, List nestedColumns) { this.aggFuncType = null; this.dataType = ClickHouseChecker.nonNull(dataType, "dataType"); 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 54d64cefa..37011e488 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 @@ -21,11 +21,14 @@ import java.time.LocalDateTime; import java.time.ZonedDateTime; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.Stack; import java.util.TimeZone; import java.util.UUID; @@ -93,18 +96,12 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce } } - ClickHouseDataType dataType = column.getDataType() == ClickHouseDataType.Dynamic ? readDynamicData() : column.getDataType(); - int estimatedLen = column.getEstimatedLength(); - int precision = column.getPrecision(); - int scale = column.getScale(); - if (dataType == ClickHouseDataType.Decimal || dataType == ClickHouseDataType.Decimal32 || - dataType == ClickHouseDataType.Decimal64 || dataType == ClickHouseDataType.Decimal128 || - dataType == ClickHouseDataType.Decimal256) { - precision = readByte(); - scale = readByte(); - System.out.println("p: " + precision + " " +scale); - } - TimeZone timezone = column.getTimeZoneOrDefault(timeZone); + ClickHouseColumn actualColumn = column.getDataType() == ClickHouseDataType.Dynamic ? readDynamicData() : column; + ClickHouseDataType dataType = actualColumn.getDataType(); + int estimatedLen = actualColumn.getEstimatedLength(); + int precision = actualColumn.getPrecision(); + int scale = actualColumn.getScale(); + TimeZone timezone = actualColumn.getTimeZoneOrDefault(timeZone); try { switch (dataType) { @@ -211,24 +208,24 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce } // case Object: // deprecated https://clickhouse.com/docs/en/sql-reference/data-types/object-data-type case Array: - return convertArray(readArray(column), typeHint); + return convertArray(readArray(actualColumn), typeHint); case Map: - return (T) readMap(column); + return (T) readMap(actualColumn); // case Nested: case Tuple: - return (T) readTuple(column); + return (T) readTuple(actualColumn); case Nothing: return null; case SimpleAggregateFunction: return (T) readValue(column.getNestedColumns().get(0)); case AggregateFunction: - return (T) readBitmap( column); + return (T) readBitmap( actualColumn); case Variant: - return (T) readVariant(column); + return (T) readVariant(actualColumn); case Dynamic: - return (T) readValue(column, typeHint); + return (T) readValue(actualColumn, typeHint); default: - throw new IllegalArgumentException("Unsupported data type: " + column.getDataType()); + throw new IllegalArgumentException("Unsupported data type: " + actualColumn.getDataType()); } } catch (EOFException e) { throw e; @@ -1003,7 +1000,15 @@ public byte[] allocate(int size) { } } - private ClickHouseDataType readDynamicData() throws IOException { + private static final Set DECIMAL_TAGS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList( + ClickHouseDataType.Decimal.getBinTag(), + ClickHouseDataType.Decimal32.getBinTag(), + ClickHouseDataType.Decimal64.getBinTag(), + ClickHouseDataType.Decimal128.getBinTag(), + ClickHouseDataType.Decimal256.getBinTag() + ))); + + private ClickHouseColumn readDynamicData() throws IOException { byte tag = readByte(); ClickHouseDataType type; @@ -1013,25 +1018,24 @@ private ClickHouseDataType readDynamicData() throws IOException { if (type == null) { throw new ClientException("Unsupported interval kind: " + intervalKind); } + return ClickHouseColumn.of("v", type, false, 0, 0); } else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) { String typeName = readString(input); - return ClickHouseDataType.valueOf(typeName); - } else if (tag == ClickHouseDataType.Decimal32.getBinTag()) { - return ClickHouseDataType.Decimal; - } else if (tag == ClickHouseDataType.Decimal64.getBinTag()) { - return ClickHouseDataType.Decimal; - } else if (tag == ClickHouseDataType.Decimal128.getBinTag()) { - return ClickHouseDataType.Decimal; - } else if (tag == ClickHouseDataType.Decimal256.getBinTag()) { - return ClickHouseDataType.Decimal; + return ClickHouseColumn.of("v", typeName); + } else if (DECIMAL_TAGS.contains(tag)) { + int precision = readByte(); + int scale = readByte(); + return ClickHouseColumn.of("v", ClickHouseDataType.binTag2Type.get(tag), false, precision, scale); + } else if (tag == ClickHouseDataType.Array.getBinTag()) { + ClickHouseColumn elementColumn = readDynamicData(); + return ClickHouseColumn.of("v", "Array(" + elementColumn.getOriginalTypeName() + ")"); } else { type = ClickHouseDataType.binTag2Type.get(tag); if (type == null) { throw new ClientException("Unsupported data type with tag " + tag); } + return ClickHouseColumn.of("v", type, false, 0, 0); } - - return type; } private static final ClickHouseColumn JSON_PLACEHOLDER_COL = ClickHouseColumn.parse("v Dynamic").get(0); 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 07287de54..ef39a6f7a 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 @@ -46,6 +46,7 @@ import java.util.TimeZone; import java.util.UUID; import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import static org.objectweb.asm.Opcodes.ACC_PUBLIC; import static org.objectweb.asm.Opcodes.ALOAD; @@ -95,7 +96,7 @@ public static void serializeData(OutputStream stream, Object value, ClickHouseCo break; case Dynamic: ClickHouseColumn typeColumn = valueToColumnForDynamicType(value); - writeDynamicTypeTag(stream, typeColumn, value); + writeDynamicTypeTag(stream, typeColumn); serializeData(stream, value, typeColumn); break; default: @@ -158,6 +159,10 @@ private static Map, ClickHouseColumn> getPredefinedTypeColumnsMap() { map.put(double[][].class, ClickHouseColumn.of("v", "Array(Array(Float64))")); map.put(double[][][].class, ClickHouseColumn.of("v", "Array(Array(Array(Float64)))")); + map.put(String[].class, ClickHouseColumn.of("v", "Array(String)")); + map.put(String[][].class, ClickHouseColumn.of("v", "Array(Array(String))")); + map.put(String[][][].class, ClickHouseColumn.of("v", "Array(Array(Array(String)))")); + return Collections.unmodifiableMap(map); } @@ -187,19 +192,45 @@ public static ClickHouseColumn valueToColumnForDynamicType(Object value) { ClickHouseColumn valueInfo = valueToColumnForDynamicType(entry.getValue()); column = ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + ", " + valueInfo.getOriginalTypeName() + ")"); } else if (value instanceof List) { + column = value2Column(value); + } else if (value == null) { + column = PREDEFINED_TYPE_COLUMNS.get(Void.class); + } else { + column = PREDEFINED_TYPE_COLUMNS.get(value.getClass()); + } + + if (column == null) { + throw new ClientException("Unable to serialize value of " + value.getClass() + " because not supported yet"); + } + + return column; + } + + // Returns null if cannot convert + // The problem here is that >2-dimensional array would require traversing all value + // to detect correct depth. Consider this example + // int[][] { + // null + // { } + // { 0, 1, 2 } + // In this case we need to find max depth. + + private static ClickHouseColumn value2Column(Object value) { + ClickHouseColumn column; + if (value instanceof List) { List list = (List) value; StringBuilder type = new StringBuilder("Array()"); - int insertPos = type.length() - 2; + int insertPos = type.length() - 1; while (!list.isEmpty() && list.get(0) instanceof List) { type.insert(insertPos, "Array()"); insertPos += 6; // add len of 'Array(' string list = (List) list.get(0); } - if (list.isEmpty()) { + if (list.isEmpty() || list.get(0) == null) { type.insert(insertPos, "Nothing"); column = ClickHouseColumn.of("v", type.toString()); } else { - ClickHouseColumn arrayBaseColumn = PREDEFINED_TYPE_COLUMNS.get(list.get(0)); + ClickHouseColumn arrayBaseColumn = PREDEFINED_TYPE_COLUMNS.get(list.get(0).getClass()); if (arrayBaseColumn != null) { type.insert(insertPos, arrayBaseColumn.getOriginalTypeName()); column = ClickHouseColumn.of("v", type.toString()); @@ -207,20 +238,13 @@ public static ClickHouseColumn valueToColumnForDynamicType(Object value) { column = null; } } - } else if (value == null) { - column = PREDEFINED_TYPE_COLUMNS.get(Void.class); } else { - column = PREDEFINED_TYPE_COLUMNS.get(value.getClass()); - } - - if (column == null) { - throw new ClientException("Unable to serialize value of " + value.getClass() + " because not supported yet"); + column = null; } - return column; } - public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typeColumn, Object value) + public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typeColumn) throws IOException { ClickHouseDataType dt = typeColumn.getDataType(); @@ -285,7 +309,8 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ break; case Array: stream.write(binTag); - // elements 0x1E + ClickHouseColumn arrayElemColumn = typeColumn.getNestedColumns().get(0); + writeDynamicTypeTag(stream, arrayElemColumn); break; case Map: stream.write(binTag); 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 f45753e68..7d7b582e3 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 @@ -410,15 +410,35 @@ public void testDynamicWithPrimitives() throws Exception { int rowId = 0; for (ClickHouseDataType dataType : ClickHouseDataType.values()) { System.out.println("Testing dynamic with " + dataType + " values"); - switch (dataType) { case Date: case Date32: case DateTime: case DateTime32: case DateTime64: + case Decimal: + case Decimal32: + case Decimal64: + case Decimal128: + case Decimal256: + // requires fix + continue; + case Array: + case Map: + case Nested: + case Tuple: + case AggregateFunction: + case SimpleAggregateFunction: + case Variant: case Enum8: case Enum16: + // tested separately + continue; + case Dynamic: + case Nothing: + case Object: + case JSON: + // no tests or tested in other tests continue; default: } @@ -480,6 +500,44 @@ public void testDynamicWithPrimitives() throws Exception { } } + @Test(groups = {"integration"}) + public void testDynamicWithArrays() throws Exception { + testDynamicWith("arrays", + new Object[]{ + "a,b", + new String[]{"a", "b"}, + Arrays.asList("c", "d") + }, + new String[]{ + "a,b", + "[a, b]", + "[c, d]" + }); + testDynamicWith("arrays", + new Object[]{ + new int[]{1, 2}, + new String[]{"a", "b"}, + Arrays.asList("c", "d") + }, + new String[]{ + "[1, 2]", + "[a, b]", + "[c, d]", + }); + + testDynamicWith("arrays", + 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]]", + "[[e, f], [j, h]]", + }); + } + @Data @AllArgsConstructor public static class DTOForDynamicPrimitivesTests { @@ -487,6 +545,31 @@ public static class DTOForDynamicPrimitivesTests { private Object field; } + private void testDynamicWith(String withWhat, Object[] values, String[] expectedStrValues) throws Exception { + if (isVersionMatch("(,24.8]")) { + return; + } + + String table = "test_dynamic_with_" + withWhat; + client.execute("DROP TABLE IF EXISTS " + table).get(); + client.execute(tableDefinition(table, "rowId Int32", "field Dynamic"), + (CommandSettings) new CommandSettings().serverSetting("enable_dynamic_type", "1")).get(); + + client.register(DTOForDynamicPrimitivesTests.class, client.getTableSchema(table)); + + List data = new ArrayList<>(); + for (int i = 0; i < values.length; i++) { + data.add(new DTOForDynamicPrimitivesTests(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")]); + } + } + private void testVariantWith(String withWhat, String[] fields, Object[] values, String[] expectedStrValues) throws Exception { if (isVersionMatch("(,24.8]")) { return; 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 index 042100504..0143a7f30 100644 --- a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java @@ -241,7 +241,7 @@ public DataTypesTestingPOJO() { 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()); - intervalYear = random.nextInt(2000, 4000); + intervalYear = random.nextInt(4000); intervalQuarter = (byte) random.nextInt(4); intervalMonth = (byte) random.nextInt(12); intervalWeek = (byte) random.nextInt(52); @@ -249,8 +249,8 @@ public DataTypesTestingPOJO() { intervalHour = (byte) random.nextInt(24); intervalMinute = (byte) random.nextInt(60); intervalSecond = (byte) random.nextInt(60); - intervalMillisecond = random.nextLong(10000); - intervalMicrosecond = random.nextLong(10000); + intervalMillisecond = random.nextLong(); + intervalMicrosecond = random.nextLong(); upper = BigInteger.valueOf(random.nextLong()).shiftLeft(64); lower = BigInteger.valueOf(random.nextLong()).and(BigInteger.valueOf(Long.MAX_VALUE)); From beae9f4465b76a3c521c0de90dfd2f1f51898a96 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Mon, 10 Feb 2025 14:23:52 -0800 Subject: [PATCH 08/21] fixed maps and decimals --- .../internal/BinaryStreamReader.java | 4 ++ .../internal/SerializerUtils.java | 32 +++++++----- .../client/datatypes/DataTypeTests.java | 51 +++++++++++++++---- 3 files changed, 63 insertions(+), 24 deletions(-) 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 37011e488..5e5045410 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 @@ -1029,6 +1029,10 @@ private ClickHouseColumn readDynamicData() throws IOException { } else if (tag == ClickHouseDataType.Array.getBinTag()) { ClickHouseColumn elementColumn = readDynamicData(); return ClickHouseColumn.of("v", "Array(" + elementColumn.getOriginalTypeName() + ")"); + } else if (tag == ClickHouseDataType.Map.getBinTag()) { + ClickHouseColumn keyInfo = readDynamicData(); + ClickHouseColumn valueInfo = readDynamicData(); + return ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + "," + valueInfo.getOriginalTypeName() + ")"); } else { type = ClickHouseDataType.binTag2Type.get(tag); if (type == null) { 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 ef39a6f7a..f230ae676 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 @@ -174,16 +174,22 @@ public static ClickHouseColumn valueToColumnForDynamicType(Object value) { } else if (value instanceof BigDecimal) { BigDecimal d = (BigDecimal) value; String decType; - if (d.scale() <= ClickHouseDataType.Decimal32.getMaxScale()) { - decType = "Decimal32"; - } else if (d.scale() <= ClickHouseDataType.Decimal64.getMaxScale()) { - decType = "Decimal64"; - } else if (d.scale() <= ClickHouseDataType.Decimal128.getMaxScale()) { + int scale; + if (d.precision() > ClickHouseDataType.Decimal128.getMaxScale()) { + decType = "Decimal256"; + scale = ClickHouseDataType.Decimal128.getMaxScale(); + } else if (d.precision() > ClickHouseDataType.Decimal64.getMaxScale()) { decType = "Decimal128"; + scale = ClickHouseDataType.Decimal128.getMaxScale(); + } else if (d.precision() > ClickHouseDataType.Decimal32.getMaxScale()) { + decType = "Decimal64"; + scale = ClickHouseDataType.Decimal128.getMaxScale(); } else { - decType = "Decimal256"; + decType = "Decimal32"; + scale = ClickHouseDataType.Decimal128.getMaxScale(); } - column = ClickHouseColumn.of("v", decType + "(" + d.precision() +"," + d.scale() + ")"); + + column = ClickHouseColumn.of("v", decType + "(" + scale + ")"); } else if (value instanceof Map) { Map map = (Map) value; // TODO: handle empty map? @@ -279,11 +285,9 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ case Decimal128: case Decimal256: stream.write(binTag); - stream.write(dt.getMaxPrecision()); - stream.write(dt.getMaxScale()); - // + BinaryStreamUtils.writeUnsignedInt8(stream, dt.getMaxPrecision()); + BinaryStreamUtils.writeUnsignedInt8(stream, dt.getMaxScale()); break; - case IntervalNanosecond: case IntervalMillisecond: case IntervalSecond: @@ -313,8 +317,10 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ writeDynamicTypeTag(stream, arrayElemColumn); break; case Map: - stream.write(binTag); - ///0x0F... + stream.write(binTag); + // 0x27 + writeDynamicTypeTag(stream, typeColumn.getKeyInfo()); + writeDynamicTypeTag(stream, typeColumn.getValueInfo()); break; case Tuple: // Tuple(T1, ..., TN) 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 7d7b582e3..233539024 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 @@ -388,8 +388,6 @@ public void testVariantWithTuple() throws Exception { }); } - - @Test(groups = {"integration"}) public void testDynamicWithPrimitives() throws Exception { @@ -416,11 +414,11 @@ public void testDynamicWithPrimitives() throws Exception { case DateTime: case DateTime32: case DateTime64: - case Decimal: - case Decimal32: - case Decimal64: - case Decimal128: - case Decimal256: +// case Decimal: +// case Decimal32: +// case Decimal64: +// case Decimal128: +// case Decimal256: // requires fix continue; case Array: @@ -486,10 +484,6 @@ public void testDynamicWithPrimitives() throws Exception { case MultiPolygon: strValue = row.getGeoMultiPolygon("field").toString(); break; - case Decimal32: - double v = row.getDouble("field"); - System.out.println(v); - break; } System.out.println("field: " + strValue + " value " + value); if (value.getClass().isPrimitive()) { @@ -538,6 +532,41 @@ public void testDynamicWithArrays() throws Exception { }); } + @Test(groups = {"integration"}) + public void testDynamicWithMaps() throws Exception { + Map map1 = new HashMap<>(); + map1.put("key1", (byte) 1); + map1.put("key2", (byte) 2); + map1.put("key3", (byte) 3); + + testDynamicWith("maps", + 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"); + + testDynamicWith("maps", + new Object[]{ + map2, + map3 + }, + new String[]{ + "{1=a, 2=b}", + "{1=a, 2=b}", + }); + } + @Data @AllArgsConstructor public static class DTOForDynamicPrimitivesTests { From 35f0b31ee2be3c92a6629017516f40f293b568fc Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 11 Feb 2025 07:45:56 -0800 Subject: [PATCH 09/21] Fixed dynamic primitives. Dates left --- .../com/clickhouse/data/ClickHouseColumn.java | 6 ++ .../clickhouse/data/ClickHouseDataType.java | 13 ++++ .../com/clickhouse/data/ClickHouseEnum.java | 14 +++- .../internal/AbstractBinaryFormatReader.java | 4 +- .../internal/BinaryStreamReader.java | 54 ++++++++++++- .../internal/SerializerUtils.java | 43 ++++++---- .../client/datatypes/DataTypeTests.java | 78 +++++++------------ .../datatypes/DataTypesTestingPOJO.java | 47 +++++++++-- 8 files changed, 188 insertions(+), 71 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 d2eb55d96..c3173d97e 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java @@ -642,6 +642,11 @@ public static List parse(String args) { public ClickHouseColumn(ClickHouseDataType dataType, String columnName, String originalTypeName, boolean nullable, boolean lowCardinality, List parameters, List nestedColumns) { + this(dataType, columnName, originalTypeName, nullable, lowCardinality, parameters, nestedColumns, ClickHouseEnum.EMPTY); + } + + public ClickHouseColumn(ClickHouseDataType dataType, String columnName, String originalTypeName, boolean nullable, + boolean lowCardinality, List parameters, List nestedColumns, ClickHouseEnum enumConstants) { this.aggFuncType = null; this.dataType = ClickHouseChecker.nonNull(dataType, "dataType"); @@ -671,6 +676,7 @@ public ClickHouseColumn(ClickHouseDataType dataType, String columnName, String o this.fixedByteLength = false; this.estimatedByteLength = 0; + this.enumConstants = enumConstants; } /** 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 a3b1ca773..aa5aa99dc 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -213,6 +213,19 @@ 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)); + + map.put(IntervalYear, setOfAllNumberClasses); + map.put(IntervalQuarter, setOfAllNumberClasses); + map.put(IntervalMonth, setOfAllNumberClasses); + map.put(IntervalWeek, setOfAllNumberClasses); + map.put(IntervalDay, setOfAllNumberClasses); + map.put(IntervalHour, setOfAllNumberClasses); + map.put(IntervalMinute, setOfAllNumberClasses); + map.put(IntervalSecond, setOfAllNumberClasses); + map.put(IntervalMillisecond, setOfAllNumberClasses); + map.put(IntervalMicrosecond, setOfAllNumberClasses); + map.put(IntervalNanosecond, setOfAllNumberClasses); + return map; } diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseEnum.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseEnum.java index 815676405..1bf3ca2a2 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseEnum.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseEnum.java @@ -51,7 +51,7 @@ protected ClickHouseEnum(Collection params) { } } - protected ClickHouseEnum(String[] names, int[] values) { + public ClickHouseEnum(String[] names, int[] values) { if (names == null || values == null) { throw new IllegalArgumentException("Non-null names and values are required"); } else if (names.length != values.length) { @@ -135,4 +135,16 @@ public String toSqlException() { } return builder.toString(); } + + public int size() { + return size; + } + + public String[] getNames() { + return names; + } + + public int[] getValues() { + return values; + } } 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 b623c6806..02a371ea2 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 @@ -291,9 +291,11 @@ public static String readAsString(Object value, ClickHouseColumn column) { return zdt.format(com.clickhouse.client.api.DataTypeUtils.DATE_FORMATTER).toString(); } return value.toString(); + } else if (value instanceof BinaryStreamReader.EnumValue) { + return ((BinaryStreamReader.EnumValue)value).name; } else if (value instanceof Number ) { ClickHouseDataType dataType = column.getDataType(); - int num = ((Number)value).intValue(); + 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 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 5e5045410..07ef5e72c 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 @@ -3,6 +3,7 @@ import com.clickhouse.client.api.ClientException; import com.clickhouse.data.ClickHouseColumn; import com.clickhouse.data.ClickHouseDataType; +import com.clickhouse.data.ClickHouseEnum; import com.clickhouse.data.value.ClickHouseBitmap; import org.slf4j.Logger; import org.slf4j.helpers.NOPLogger; @@ -158,9 +159,11 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce case Bool: return (T) Boolean.valueOf(readByteOrEOF(input) == 1); case Enum8: - return (T) Byte.valueOf((byte) readUnsignedByte()); + byte enum8Val = (byte) readUnsignedByte(); + return (T) new EnumValue(actualColumn.getEnumConstants().name(enum8Val), enum8Val); case Enum16: - return (T) Short.valueOf((short) readUnsignedShortLE()); + short enum16Val = (short) readUnsignedShortLE(); + return (T) new EnumValue(actualColumn.getEnumConstants().name(enum16Val), enum16Val); case Date: return convertDateTime(readDate(timezone), typeHint); case Date32: @@ -684,6 +687,38 @@ public synchronized List asList() { } } + public static class EnumValue extends Number { + + public final String name; + + public final int value; + + public EnumValue(String name, int value) { + this.name = name; + this.value = value; + } + + @Override + public int intValue() { + return value; + } + + @Override + public long longValue() { + return value; + } + + @Override + public float floatValue() { + return value; + } + + @Override + public double doubleValue() { + return value; + } + } + /** * Reads a map. * @param column - column information @@ -1033,6 +1068,21 @@ private ClickHouseColumn readDynamicData() throws IOException { ClickHouseColumn keyInfo = readDynamicData(); ClickHouseColumn valueInfo = readDynamicData(); return ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + "," + valueInfo.getOriginalTypeName() + ")"); + } else if (tag == ClickHouseDataType.Enum8.getBinTag() || tag == ClickHouseDataType.Enum16.getBinTag()) { + int constants = readVarInt(input); + int []values = new int[constants]; + String []names = new String[constants]; + ClickHouseDataType enumType = constants > 127 ? ClickHouseDataType.Enum16 : ClickHouseDataType.Enum8; + for (int i = 0; i < constants; i++) { + names[i] = readString(input); + if (enumType == ClickHouseDataType.Enum8) { + values[i] = readUnsignedByte(); + } else { + values[i] = readUnsignedShortLE(); + } + } + return new ClickHouseColumn(enumType, "v", enumType.name(), false, false, Collections.emptyList(), Collections.emptyList(), + new ClickHouseEnum(names, values)); } else { type = ClickHouseDataType.binTag2Type.get(tag); if (type == null) { 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 f230ae676..1131a439b 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 @@ -6,6 +6,7 @@ import com.clickhouse.data.ClickHouseAggregateFunction; import com.clickhouse.data.ClickHouseColumn; import com.clickhouse.data.ClickHouseDataType; +import com.clickhouse.data.ClickHouseEnum; import com.clickhouse.data.format.BinaryStreamUtils; import com.clickhouse.data.value.ClickHouseBitmap; import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; @@ -28,12 +29,10 @@ import java.net.Inet4Address; import java.net.Inet6Address; import java.sql.Timestamp; -import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.ZoneId; import java.time.ZonedDateTime; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -46,7 +45,6 @@ import java.util.TimeZone; import java.util.UUID; import java.util.stream.Collectors; -import java.util.stream.StreamSupport; import static org.objectweb.asm.Opcodes.ACC_PUBLIC; import static org.objectweb.asm.Opcodes.ALOAD; @@ -177,16 +175,16 @@ public static ClickHouseColumn valueToColumnForDynamicType(Object value) { int scale; if (d.precision() > ClickHouseDataType.Decimal128.getMaxScale()) { decType = "Decimal256"; - scale = ClickHouseDataType.Decimal128.getMaxScale(); + scale = ClickHouseDataType.Decimal256.getMaxScale(); } else if (d.precision() > ClickHouseDataType.Decimal64.getMaxScale()) { decType = "Decimal128"; scale = ClickHouseDataType.Decimal128.getMaxScale(); } else if (d.precision() > ClickHouseDataType.Decimal32.getMaxScale()) { decType = "Decimal64"; - scale = ClickHouseDataType.Decimal128.getMaxScale(); + scale = ClickHouseDataType.Decimal64.getMaxScale(); } else { decType = "Decimal32"; - scale = ClickHouseDataType.Decimal128.getMaxScale(); + scale = ClickHouseDataType.Decimal32.getMaxScale(); } column = ClickHouseColumn.of("v", decType + "(" + scale + ")"); @@ -197,8 +195,10 @@ public static ClickHouseColumn valueToColumnForDynamicType(Object value) { ClickHouseColumn keyInfo = valueToColumnForDynamicType(entry.getKey()); ClickHouseColumn valueInfo = valueToColumnForDynamicType(entry.getValue()); column = ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + ", " + valueInfo.getOriginalTypeName() + ")"); + } else if (value instanceof Enum) { + column = enumValue2Column((Enum)value); } else if (value instanceof List) { - column = value2Column(value); + column = listValue2Column(value); } else if (value == null) { column = PREDEFINED_TYPE_COLUMNS.get(Void.class); } else { @@ -221,7 +221,7 @@ public static ClickHouseColumn valueToColumnForDynamicType(Object value) { // { 0, 1, 2 } // In this case we need to find max depth. - private static ClickHouseColumn value2Column(Object value) { + private static ClickHouseColumn listValue2Column(Object value) { ClickHouseColumn column; if (value instanceof List) { List list = (List) value; @@ -250,6 +250,12 @@ private static ClickHouseColumn value2Column(Object value) { return column; } + private static ClickHouseColumn enumValue2Column(Enum enumValue) { + ClickHouseEnum clickHouseEnum= ClickHouseEnum.of(enumValue.getClass()); + return new ClickHouseColumn(clickHouseEnum.size() > 127 ? ClickHouseDataType.Enum16 : ClickHouseDataType.Enum8, "v", "Enum16", false, false, Collections.emptyList(), Collections.emptyList(), + clickHouseEnum); + } + public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typeColumn) throws IOException { @@ -272,12 +278,20 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ writeVarInt(stream, typeColumn.getEstimatedLength()); break; case Enum8: - stream.write(binTag); - /// 0x17... - break; case Enum16: stream.write(binTag); - //0x18...> + ClickHouseEnum enumVal = typeColumn.getEnumConstants(); + String[] names = enumVal.getNames(); + int[] values = enumVal.getValues(); + writeVarInt(stream, names.length); + for (int i = 0; i < enumVal.size(); i++ ) { + BinaryStreamUtils.writeString(stream, names[i]); + if (dt == ClickHouseDataType.Enum8) { + BinaryStreamUtils.writeInt8(stream, values[i]); + } else { + BinaryStreamUtils.writeInt16(stream, values[i]); + } + } break; case Decimal: case Decimal32: @@ -524,6 +538,7 @@ private static void serializePrimitiveData(OutputStream stream, Object value, Cl serializeJSON(stream, value); break; case IntervalNanosecond: + case IntervalMicrosecond: case IntervalMillisecond: case IntervalSecond: case IntervalMinute: @@ -545,7 +560,9 @@ private static void serializeEnumData(OutputStream stream, ClickHouseColumn colu if (value instanceof String) { enumValue = column.getEnumConstants().value((String) value); } else if (value instanceof Number) { - enumValue = ((Number)value).intValue(); + enumValue = ((Number) value).intValue(); + } else if (value instanceof Enum) { + enumValue = ((Enum)value).ordinal(); } else { throw new IllegalArgumentException("Cannot write value of class " + value.getClass() + " into column with Enum type " + column.getOriginalTypeName()); } 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 233539024..eb185249e 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,9 +6,7 @@ import com.clickhouse.client.ClickHouseServerForTest; 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; @@ -24,17 +22,16 @@ import java.io.IOException; import java.lang.reflect.Method; +import java.math.BigDecimal; import java.time.LocalDateTime; import java.time.temporal.ChronoUnit; 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; public class DataTypeTests extends BaseIntegrationTest { @@ -63,7 +60,6 @@ public void setUp() throws IOException { .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), false) .setUsername("default") .setPassword(ClickHouseServerForTest.getPassword()) - .useNewImplementation(System.getProperty("client.tests.useNewImplementation", "true").equals("true")) .compressClientRequest(useClientCompression) .useHttpCompression(useHttpCompression) .build(); @@ -115,11 +111,8 @@ public void testArrays() throws Exception { 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()); }); } @@ -153,7 +146,6 @@ public void testVariantWithSimpleDataTypes() throws Exception { 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()); @@ -161,21 +153,11 @@ public void testVariantWithSimpleDataTypes() throws Exception { 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: + case Dynamic: // skipped continue dataTypesLoop; @@ -205,10 +187,10 @@ public void testVariantWithSimpleDataTypes() throws Exception { 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; } } + Assert.assertNotNull(value); List data = new ArrayList<>(); data.add(new DTOForVariantPrimitivesTests(0, value)); @@ -241,7 +223,6 @@ public void testVariantWithSimpleDataTypes() throws Exception { strValue = row.getGeoMultiPolygon("field").toString(); break; } - System.out.println("field: " + strValue + " value " + value); if (value.getClass().isPrimitive()) { Assert.assertEquals(strValue, String.valueOf(value)); } else { @@ -407,57 +388,52 @@ public void testDynamicWithPrimitives() throws Exception { int rowId = 0; for (ClickHouseDataType dataType : ClickHouseDataType.values()) { - System.out.println("Testing dynamic with " + dataType + " values"); switch (dataType) { case Date: case Date32: case DateTime: case DateTime32: case DateTime64: -// case Decimal: -// case Decimal32: -// case Decimal64: -// case Decimal128: -// case Decimal256: // requires fix continue; case Array: case Map: - case Nested: - case Tuple: case AggregateFunction: case SimpleAggregateFunction: - case Variant: - case Enum8: - case Enum16: // tested separately continue; case Dynamic: - case Nothing: - case Object: + case Nothing: // array tests + case Object: // deprecated case JSON: + case Nested: + case Tuple: + case Variant: + case Decimal: // virtual type // no tests or tested in other tests continue; default: } 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; + if (dataType == ClickHouseDataType.Enum8) { + value = sample.getSmallEnum(); + } else if (dataType == ClickHouseDataType.Enum16) { + value = sample.getLargeEnum(); + } else { + for (Method m : sample.getClass().getDeclaredMethods()) { + if (m.getName().equalsIgnoreCase("get" + dataType.name())) { + value = m.invoke(sample); + break; + } } } + Assert.assertNotNull(value); + List data = new ArrayList<>(); data.add(new DTOForDynamicPrimitivesTests(rowId++, value)); - try { - client.insert(table, data).get().close(); - } catch (Exception e) { - System.out.println("Failed for " + dataType + ": " + e.getMessage()); - continue; - } + client.insert(table, data).get().close(); List rows = client.queryAll("SELECT * FROM " + table + " ORDER BY rowId DESC "); GenericRecord row = rows.get(0); String strValue = row.getString("field"); @@ -484,8 +460,14 @@ public void testDynamicWithPrimitives() throws Exception { case MultiPolygon: strValue = row.getGeoMultiPolygon("field").toString(); break; + case Decimal32: + case Decimal64: + case Decimal128: + case Decimal256: + BigDecimal tmpDec = row.getBigDecimal("field").stripTrailingZeros(); + strValue = tmpDec.toPlainString(); + break; } - System.out.println("field: " + strValue + " value " + value); if (value.getClass().isPrimitive()) { Assert.assertEquals(strValue, String.valueOf(value)); } else { @@ -594,7 +576,6 @@ private void testDynamicWith(String withWhat, Object[] values, String[] expected 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")]); } } @@ -621,7 +602,6 @@ private void testVariantWith(String withWhat, String[] fields, Object[] values, 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")]); } } 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 index 0143a7f30..cae20b008 100644 --- a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java @@ -116,6 +116,10 @@ public class DataTypesTestingPOJO { private BigInteger intervalNanosecond; + private SmallEnum smallEnum; + + private LargeEnum largeEnum; + public DataTypesTestingPOJO() { final Random random = new Random(); byteValue = (byte) random.nextInt(); @@ -249,13 +253,13 @@ public DataTypesTestingPOJO() { intervalHour = (byte) random.nextInt(24); intervalMinute = (byte) random.nextInt(60); intervalSecond = (byte) random.nextInt(60); - intervalMillisecond = random.nextLong(); - intervalMicrosecond = random.nextLong(); + intervalMillisecond = Math.abs(random.nextLong()); + intervalMicrosecond = Math.abs(random.nextInt()); - upper = BigInteger.valueOf(random.nextLong()).shiftLeft(64); - lower = BigInteger.valueOf(random.nextLong()).and(BigInteger.valueOf(Long.MAX_VALUE)); + intervalNanosecond = BigInteger.valueOf(Math.abs(random.nextLong())); - intervalNanosecond = upper.or(lower); + smallEnum = SmallEnum.valueOf("CONSTANT_" + Math.max(1, random.nextInt(SmallEnum.values().length - 1))); + largeEnum = LargeEnum.valueOf("CONSTANT_" + Math.max(1, random.nextInt(LargeEnum.values().length - 1))); } public boolean getBool() { @@ -328,4 +332,37 @@ public static String generateTableCreateSQL(String tableName) { "intervalNanosecond IntervalNanosecond " + ") ENGINE = MergeTree ORDER BY ()"; } + + public enum SmallEnum { + CONSTANT_1, CONSTANT_2, CONSTANT_3, CONSTANT_4, CONSTANT_5, CONSTANT_6, CONSTANT_7, CONSTANT_8, CONSTANT_9, CONSTANT_10; + } + + public enum LargeEnum { + CONSTANT_1, CONSTANT_2, CONSTANT_3, CONSTANT_4, CONSTANT_5, CONSTANT_6, CONSTANT_7, CONSTANT_8, CONSTANT_9, CONSTANT_10, + CONSTANT_11, CONSTANT_12, CONSTANT_13, CONSTANT_14, CONSTANT_15, CONSTANT_16, CONSTANT_17, CONSTANT_18, CONSTANT_19, CONSTANT_20, + CONSTANT_21, CONSTANT_22, CONSTANT_23, CONSTANT_24, CONSTANT_25, CONSTANT_26, CONSTANT_27, CONSTANT_28, CONSTANT_29, CONSTANT_30, + CONSTANT_31, CONSTANT_32, CONSTANT_33, CONSTANT_34, CONSTANT_35, CONSTANT_36, CONSTANT_37, CONSTANT_38, CONSTANT_39, CONSTANT_40, + CONSTANT_41, CONSTANT_42, CONSTANT_43, CONSTANT_44, CONSTANT_45, CONSTANT_46, CONSTANT_47, CONSTANT_48, CONSTANT_49, CONSTANT_50, + CONSTANT_51, CONSTANT_52, CONSTANT_53, CONSTANT_54, CONSTANT_55, CONSTANT_56, CONSTANT_57, CONSTANT_58, CONSTANT_59, CONSTANT_60, + CONSTANT_61, CONSTANT_62, CONSTANT_63, CONSTANT_64, CONSTANT_65, CONSTANT_66, CONSTANT_67, CONSTANT_68, CONSTANT_69, CONSTANT_70, + CONSTANT_71, CONSTANT_72, CONSTANT_73, CONSTANT_74, CONSTANT_75, CONSTANT_76, CONSTANT_77, CONSTANT_78, CONSTANT_79, CONSTANT_80, + CONSTANT_81, CONSTANT_82, CONSTANT_83, CONSTANT_84, CONSTANT_85, CONSTANT_86, CONSTANT_87, CONSTANT_88, CONSTANT_89, CONSTANT_90, + CONSTANT_91, CONSTANT_92, CONSTANT_93, CONSTANT_94, CONSTANT_95, CONSTANT_96, CONSTANT_97, CONSTANT_98, CONSTANT_99, CONSTANT_100, + CONSTANT_101, CONSTANT_102, CONSTANT_103, CONSTANT_104, CONSTANT_105, CONSTANT_106, CONSTANT_107, CONSTANT_108, CONSTANT_109, CONSTANT_110, + CONSTANT_111, CONSTANT_112, CONSTANT_113, CONSTANT_114, CONSTANT_115, CONSTANT_116, CONSTANT_117, CONSTANT_118, CONSTANT_119, CONSTANT_120, + CONSTANT_121, CONSTANT_122, CONSTANT_123, CONSTANT_124, CONSTANT_125, CONSTANT_126, CONSTANT_127, CONSTANT_128, CONSTANT_129, CONSTANT_130, + CONSTANT_131, CONSTANT_132, CONSTANT_133, CONSTANT_134, CONSTANT_135, CONSTANT_136, CONSTANT_137, CONSTANT_138, CONSTANT_139, CONSTANT_140, + CONSTANT_141, CONSTANT_142, CONSTANT_143, CONSTANT_144, CONSTANT_145, CONSTANT_146, CONSTANT_147, CONSTANT_148, CONSTANT_149, CONSTANT_150, + CONSTANT_151, CONSTANT_152, CONSTANT_153, CONSTANT_154, CONSTANT_155, CONSTANT_156, CONSTANT_157, CONSTANT_158, CONSTANT_159, CONSTANT_160, + CONSTANT_161, CONSTANT_162, CONSTANT_163, CONSTANT_164, CONSTANT_165, CONSTANT_166, CONSTANT_167, CONSTANT_168, CONSTANT_169, CONSTANT_170, + CONSTANT_171, CONSTANT_172, CONSTANT_173, CONSTANT_174, CONSTANT_175, CONSTANT_176, CONSTANT_177, CONSTANT_178, CONSTANT_179, CONSTANT_180, + CONSTANT_181, CONSTANT_182, CONSTANT_183, CONSTANT_184, CONSTANT_185, CONSTANT_186, CONSTANT_187, CONSTANT_188, CONSTANT_189, CONSTANT_190, + CONSTANT_191, CONSTANT_192, CONSTANT_193, CONSTANT_194, CONSTANT_195, CONSTANT_196, CONSTANT_197, CONSTANT_198, CONSTANT_199, CONSTANT_200, + CONSTANT_201, CONSTANT_202, CONSTANT_203, CONSTANT_204, CONSTANT_205, CONSTANT_206, CONSTANT_207, CONSTANT_208, CONSTANT_209, CONSTANT_210, + CONSTANT_211, CONSTANT_212, CONSTANT_213, CONSTANT_214, CONSTANT_215, CONSTANT_216, CONSTANT_217, CONSTANT_218, CONSTANT_219, CONSTANT_220, + CONSTANT_221, CONSTANT_222, CONSTANT_223, CONSTANT_224, CONSTANT_225, CONSTANT_226, CONSTANT_227, CONSTANT_228, CONSTANT_229, CONSTANT_230, + CONSTANT_231, CONSTANT_232, CONSTANT_233, CONSTANT_234, CONSTANT_235, CONSTANT_236, CONSTANT_237, CONSTANT_238, CONSTANT_239, CONSTANT_240, + CONSTANT_241, CONSTANT_242, CONSTANT_243, CONSTANT_244, CONSTANT_245, CONSTANT_246, CONSTANT_247, CONSTANT_248, CONSTANT_249, CONSTANT_250, + CONSTANT_251, CONSTANT_252, CONSTANT_253, CONSTANT_254, CONSTANT_255, CONSTANT_256, CONSTANT_257, CONSTANT_258, CONSTANT_259, CONSTANT_260 + } } From c6335d7dfbe3c25b1bb0b7fe25dfa889d3f1331b Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 11 Feb 2025 10:42:00 -0800 Subject: [PATCH 10/21] fixed enums for different releases --- .../internal/AbstractBinaryFormatReader.java | 10 ++++++---- .../com/clickhouse/client/datatypes/DataTypeTests.java | 2 +- .../java/com/clickhouse/client/query/QueryTests.java | 2 +- 3 files changed, 8 insertions(+), 6 deletions(-) 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 02a371ea2..07c030f60 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 @@ -690,22 +690,24 @@ public Object[] getTuple(String colName) { @Override public byte getEnum8(String colName) { - return readValue(colName); + BinaryStreamReader.EnumValue enumValue = readValue(colName); + return enumValue.byteValue(); } @Override public byte getEnum8(int index) { - return readValue(index); + return getEnum8(schema.columnIndexToName(index)); } @Override public short getEnum16(String colName) { - return readValue(colName); + BinaryStreamReader.EnumValue enumValue = readValue(colName); + return enumValue.shortValue(); } @Override public short getEnum16(int index) { - return readValue(index); + return getEnum16(schema.columnIndexToName(index)); } @Override 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 eb185249e..026b43f9a 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 @@ -383,7 +383,7 @@ public void testDynamicWithPrimitives() throws Exception { String createTableStatement = " CREATE TABLE " + table + "( rowId Int64, field Dynamic ) " + "Engine = MergeTree ORDER BY ()"; - client.execute(createTableStatement, (CommandSettings) new CommandSettings().serverSetting("enable_dynamic_type", "1")); + client.execute(createTableStatement, (CommandSettings) new CommandSettings().serverSetting("allow_experimental_dynamic_type", "1")); client.register(DTOForDynamicPrimitivesTests.class, client.getTableSchema(table)); int rowId = 0; 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 52b618603..93fb0f9b1 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 @@ -1420,7 +1420,7 @@ private List> prepareDataSet(String table, List colu // Create table CommandSettings settings = new CommandSettings(); if (isVersionMatch("[24.8,)")) { - settings.serverSetting("enable_dynamic_type", "1") + settings.serverSetting("allow_experimental_dynamic_type", "1") .serverSetting("allow_experimental_json_type", "1"); } StringBuilder createStmtBuilder = new StringBuilder(); From dbee7220bfddd3190c9c45f3fca59abe01a0c1a1 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 11 Feb 2025 16:21:30 -0800 Subject: [PATCH 11/21] fixed arrays --- .../internal/BinaryStreamReader.java | 40 ++----- .../internal/SerializerUtils.java | 113 ++++++++++-------- .../client/datatypes/DataTypeTests.java | 37 +++--- 3 files changed, 84 insertions(+), 106 deletions(-) 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 07ef5e72c..a90f80af2 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 @@ -527,36 +527,7 @@ public static byte[] readNBytesLE(InputStream input, byte[] buffer, int offset, return bytes; } - - public ArrayValue readArrayStack(int levels, ClickHouseColumn baseElementColumn) throws IOException { - - Stack arrays = new Stack<>(); - int level = levels; - ArrayValue array = null; - while (level <= levels) { - if (level != 1 && arrays.size() < level) { - int len = readVarInt(input); - arrays.push(new ArrayValue(ArrayValue.class, len)); - level--; - } else if (level == 1) { - int len = readVarInt(input); - array = readArrayItem(baseElementColumn, len); - level++; - } else if (array !=null) { // some array read completely - ArrayValue tmp = arrays.pop(); - if (tmp.append(array)) { // array filled - array = tmp; - level++; - } else { - array = null; - level--; - } - } - } - - return array; - } - + /** * Reads a array into an ArrayValue object. * @param column - column information @@ -1070,8 +1041,8 @@ private ClickHouseColumn readDynamicData() throws IOException { return ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + "," + valueInfo.getOriginalTypeName() + ")"); } else if (tag == ClickHouseDataType.Enum8.getBinTag() || tag == ClickHouseDataType.Enum16.getBinTag()) { int constants = readVarInt(input); - int []values = new int[constants]; - String []names = new String[constants]; + int[] values = new int[constants]; + String[] names = new String[constants]; ClickHouseDataType enumType = constants > 127 ? ClickHouseDataType.Enum16 : ClickHouseDataType.Enum8; for (int i = 0; i < constants; i++) { names[i] = readString(input); @@ -1081,8 +1052,11 @@ private ClickHouseColumn readDynamicData() throws IOException { values[i] = readUnsignedShortLE(); } } - return new ClickHouseColumn(enumType, "v", enumType.name(), false, false, Collections.emptyList(), Collections.emptyList(), + return new ClickHouseColumn(enumType, "v", enumType.name(), false, false, Collections.emptyList(), Collections.emptyList(), new ClickHouseEnum(names, values)); + } else if (tag == ClickHouseDataType.NULLABLE_BIN_TAG) { + ClickHouseColumn column = readDynamicData(); + return ClickHouseColumn.of("v", "Nullable(" + column.getOriginalTypeName() + ")"); } else { type = ClickHouseDataType.binTag2Type.get(tag); if (type == null) { 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 038b69032..deafc66f0 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 @@ -38,6 +38,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.Stack; import java.util.StringTokenizer; import java.util.TimeZone; import java.util.UUID; @@ -154,10 +155,6 @@ private static Map, ClickHouseColumn> getPredefinedTypeColumnsMap() { map.put(double[][].class, ClickHouseColumn.of("v", "Array(Array(Float64))")); map.put(double[][][].class, ClickHouseColumn.of("v", "Array(Array(Array(Float64)))")); - map.put(String[].class, ClickHouseColumn.of("v", "Array(String)")); - map.put(String[][].class, ClickHouseColumn.of("v", "Array(Array(String))")); - map.put(String[][][].class, ClickHouseColumn.of("v", "Array(Array(Array(String)))")); - return Collections.unmodifiableMap(map); } @@ -194,7 +191,7 @@ public static ClickHouseColumn valueToColumnForDynamicType(Object value) { column = ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + ", " + valueInfo.getOriginalTypeName() + ")"); } else if (value instanceof Enum) { column = enumValue2Column((Enum)value); - } else if (value instanceof List) { + } else if (value instanceof List || (value !=null && value.getClass().isArray())) { column = listValue2Column(value); } else if (value == null) { column = PREDEFINED_TYPE_COLUMNS.get(Void.class); @@ -219,28 +216,57 @@ public static ClickHouseColumn valueToColumnForDynamicType(Object value) { // In this case we need to find max depth. private static ClickHouseColumn listValue2Column(Object value) { - ClickHouseColumn column; - if (value instanceof List) { - List list = (List) value; - StringBuilder type = new StringBuilder("Array()"); - int insertPos = type.length() - 1; - while (!list.isEmpty() && list.get(0) instanceof List) { - type.insert(insertPos, "Array()"); - insertPos += 6; // add len of 'Array(' string - list = (List) list.get(0); - } - if (list.isEmpty() || list.get(0) == null) { - type.insert(insertPos, "Nothing"); - column = ClickHouseColumn.of("v", type.toString()); - } else { - ClickHouseColumn arrayBaseColumn = PREDEFINED_TYPE_COLUMNS.get(list.get(0).getClass()); + + ClickHouseColumn column = PREDEFINED_TYPE_COLUMNS.get(value.getClass()); + if (column != null) { + return column; + } + + if (value instanceof List || (value.getClass().isArray())) { + Stack arrays = new Stack<>(); + arrays.push(new Object[]{value, 1}); + int maxDepth = 0; + boolean hasNulls = false; + ClickHouseColumn arrayBaseColumn = null; + StringBuilder typeStr = new StringBuilder(); + int insertPos = 0; + while (!arrays.isEmpty()) { + Object[] arr = arrays.pop(); + int depth = (Integer) arr[1]; + if (depth > maxDepth) { + maxDepth = depth; + typeStr.insert(insertPos, "Array()"); + insertPos += 6; + } + + boolean isArray = arr[0].getClass().isArray(); + List list = isArray ? null : ((List) arr[0]); + int len = isArray ? Array.getLength(arr[0]) : list.size(); + for (int i = 0; i < len; i++) { + Object item = isArray ? Array.get(arr[0], i) : list.get(i); + if (!hasNulls && item == null) { + hasNulls = true; + } else if (item != null && (item instanceof List || item.getClass().isArray())) { + arrays.push(new Object[]{item, depth + 1}); + } else if (arrayBaseColumn == null && item != null) { + arrayBaseColumn = PREDEFINED_TYPE_COLUMNS.get(item.getClass()); + if (arrayBaseColumn == null) { + throw new ClientException("Cannot serialize " + item.getClass() + " as array element"); + } + } + } + if (arrayBaseColumn != null) { - type.insert(insertPos, arrayBaseColumn.getOriginalTypeName()); - column = ClickHouseColumn.of("v", type.toString()); - } else { - column = null; + if (hasNulls) { + typeStr.insert(insertPos, "Nullable()"); + insertPos += 9; + } + typeStr.insert(insertPos, arrayBaseColumn.getOriginalTypeName()); + break; } } + + column = ClickHouseColumn.of("v", typeStr.toString()); } else { column = null; } @@ -370,27 +396,20 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ } public static void serializeArrayData(OutputStream stream, Object value, ClickHouseColumn column) throws IOException { + if (value == null) { + writeVarInt(stream, 0); + return; + } - 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.getNestedColumns().get(0)); - } - } 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()) { + boolean isArray = value.getClass().isArray(); + if (value instanceof List || isArray) { + List list = isArray ? null : (List)value; + int len = isArray ? Array.getLength(value) : list.size(); + + writeVarInt(stream, len); + for (int i = 0; i < len; i++) { + Object val = isArray? Array.get(value, i) : list.get(i); + if (column.getArrayNestedLevel() == 1 && column.getArrayBaseColumn().isNullable()) { if (val == null) { writeNull(stream); continue; @@ -515,12 +534,6 @@ 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: case Enum16: serializeEnumData(stream, column, 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 026b43f9a..760d5945e 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,6 +27,7 @@ import java.time.temporal.ChronoUnit; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -389,13 +390,6 @@ public void testDynamicWithPrimitives() throws Exception { int rowId = 0; for (ClickHouseDataType dataType : ClickHouseDataType.values()) { switch (dataType) { - case Date: - case Date32: - case DateTime: - case DateTime32: - case DateTime64: - // requires fix - continue; case Array: case Map: case AggregateFunction: @@ -481,36 +475,33 @@ public void testDynamicWithArrays() throws Exception { testDynamicWith("arrays", new Object[]{ "a,b", - new String[]{"a", "b"}, - Arrays.asList("c", "d") + new String[]{"a", null, "b"}, + Arrays.asList("c", "d"), + new Integer[]{1, null, 2, null, 3} + }, new String[]{ "a,b", - "[a, b]", - "[c, d]" + "[a, null, b]", + "[c, d]", + "[1, null, 2, null, 3]" }); testDynamicWith("arrays", new Object[]{ new int[]{1, 2}, new String[]{"a", "b"}, - Arrays.asList("c", "d") + Arrays.asList("c", "d"), + Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4)), + Arrays.asList(Arrays.asList(1, 2), Collections.emptyList()), + Arrays.asList(Arrays.asList(1, 2), null, Arrays.asList(3, 4)) }, new String[]{ "[1, 2]", "[a, b]", "[c, d]", - }); - - testDynamicWith("arrays", - 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]]", - "[[e, f], [j, h]]", + "[[1, 2], []]", + "[[1, 2], [], [3, 4]]" }); } From 455b3678162ee6a3fe0e2cdcee86c24c6ae52709 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 12 Feb 2025 10:44:38 -0800 Subject: [PATCH 12/21] enabled JDBC tests for variant and dynamic --- .../com/clickhouse/jdbc/DataTypeTests.java | 54 +++++++++++++++++-- .../clickhouse/jdbc/JdbcIntegrationTest.java | 6 ++- 2 files changed, 56 insertions(+), 4 deletions(-) diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java index 2e2627076..be11457b5 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java @@ -1,5 +1,6 @@ package com.clickhouse.jdbc; +import com.clickhouse.client.api.ClientConfigProperties; import com.clickhouse.data.Tuple; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -24,6 +25,7 @@ import java.util.GregorianCalendar; import java.util.HashMap; import java.util.Map; +import java.util.Properties; import java.util.Random; import java.util.TimeZone; import java.util.UUID; @@ -778,8 +780,9 @@ public void testJSONTypeSimpleStatement() throws SQLException { - @Test (enabled = false)//TODO: The client doesn't support all of these yet + @Test(groups = { "integration" }, enabled = false) public void testGeometricTypesSimpleStatement() throws SQLException { + // TODO: add LineString and MultiLineString support runQuery("CREATE TABLE test_geometric (order Int8, " + "point Point, ring Ring, linestring LineString, multilinestring MultiLineString, polygon Polygon, multipolygon MultiPolygon" + ") ENGINE = MergeTree ORDER BY ()"); @@ -818,11 +821,14 @@ public void testGeometricTypesSimpleStatement() throws SQLException { } - @Test (enabled = false)//TODO: This type is experimental right now + @Test(groups = { "integration" }) public void testDynamicTypesSimpleStatement() throws SQLException { + Properties properties = new Properties(); + properties.setProperty(ClientConfigProperties.serverSetting("allow_experimental_dynamic_type"), "1"); runQuery("CREATE TABLE test_dynamic (order Int8, " + "dynamic Dynamic" - + ") ENGINE = MergeTree ORDER BY ()"); + + ") ENGINE = MergeTree ORDER BY ()", + properties); // Insert random (valid) values long seed = System.currentTimeMillis(); @@ -926,4 +932,46 @@ public void testTypeConversions() throws Exception { } } } + + @Test(groups = { "integration" }) + public void testVariantTypesSimpleStatement() throws SQLException { + Properties properties = new Properties(); + properties.setProperty(ClientConfigProperties.serverSetting("allow_experimental_variant_type"), "1"); + runQuery("CREATE TABLE test_variant (order Int8, " + + "v Variant(String, Int32)" + + ") ENGINE = MergeTree ORDER BY ()", + properties); + + // Insert random (valid) values + long seed = System.currentTimeMillis(); + Random rand = new Random(seed); + log.info("Random seed was: {}", seed); + + String variant1 = "string" + rand.nextInt(1000); + int variant2 = rand.nextInt(256) - 128; + + String sql = String.format("INSERT INTO test_variant VALUES ( 1, '%s' )", variant1); + insertData(sql); + + sql = String.format("INSERT INTO test_variant VALUES ( 2, %d )", variant2); + insertData(sql); + + + // Check the results + try (Connection conn = getConnection()) { + try (Statement stmt = conn.createStatement()) { + try (ResultSet rs = stmt.executeQuery("SELECT * FROM test_variant ORDER BY order")) { + assertTrue(rs.next()); + assertEquals(rs.getString("v"), variant1); + assertTrue(rs.getObject("v") instanceof String); + + assertTrue(rs.next()); + assertEquals(rs.getInt("v"), variant2); + assertTrue(rs.getObject("v") instanceof Number); + + assertFalse(rs.next()); + } + } + } + } } diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java index 93000c56d..af42aada7 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java @@ -49,7 +49,11 @@ protected static String getDatabase() { } protected boolean runQuery(String query) { - try (Connection connection = getJdbcConnection()) { + return runQuery(query, new Properties()); + } + + protected boolean runQuery(String query, Properties connProperties) { + try (Connection connection = getJdbcConnection(connProperties)) { try (Statement stmt = connection.createStatement()) { return stmt.execute(query); } From de27d86cf7dd276de1f5772a9186f72aa16b524e Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 12 Feb 2025 16:35:55 -0800 Subject: [PATCH 13/21] fixed dates, added test for POJO read --- .../clickhouse/data/ClickHouseDataType.java | 2 +- .../RowBinaryFormatSerializer.java | 4 ++ .../internal/BinaryStreamReader.java | 25 ++++++++-- .../internal/SerializerUtils.java | 15 ++++-- .../api/internal/HttpAPIClientHelper.java | 2 +- .../clickhouse/client/insert/InsertTests.java | 50 +++++++++++++++++-- .../client/insert/PojoWithDynamic.java | 29 +++++++++++ .../clickhouse/client/query/QueryTests.java | 4 +- 8 files changed, 113 insertions(+), 18 deletions(-) create mode 100644 client-v2/src/test/java/com/clickhouse/client/insert/PojoWithDynamic.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 aa5aa99dc..86e196bc2 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -54,7 +54,7 @@ public enum ClickHouseDataType { Date32(LocalDate.class, false, false, false, 4, 10, 0, 0, 0, false, 0x10), DateTime(LocalDateTime.class, true, false, false, 0, 29, 0, 0, 9, false, 0x11, "TIMESTAMP"), DateTime32(LocalDateTime.class, true, false, false, 4, 19, 0, 0, 0, false, 0x12), - DateTime64(LocalDateTime.class, true, false, false, 8, 29, 3, 0, 9, false, 0x13), + DateTime64(LocalDateTime.class, true, false, false, 8, 29, 3, 0, 9, false, 0x14), // we always write timezone as argument Enum8(String.class, true, true, false, 1, 0, 0, 0, 0, false, 0x17, "ENUM"), Enum16(String.class, true, true, false, 2, 0, 0, 0, 0, false, 0x18), FixedString(String.class, true, true, false, 0, 0, 0, 0, 0, false, 0x16, "BINARY"), diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryFormatSerializer.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryFormatSerializer.java index 9dc045053..25b170165 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryFormatSerializer.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryFormatSerializer.java @@ -207,6 +207,8 @@ public static boolean writeValuePreamble(OutputStream out, boolean defaultsSuppo return false;//And we're done } else if (dataType == ClickHouseDataType.Array) {//If the column is an array SerializerUtils.writeNonNull(out);//Then we send nonNull + } else if (dataType == ClickHouseDataType.Dynamic) { + // do nothing } else { throw new IllegalArgumentException(String.format("An attempt to write null into not nullable column '%s'", column)); } @@ -221,6 +223,8 @@ public static boolean writeValuePreamble(OutputStream out, boolean defaultsSuppo } else if (value == null) { if (dataType == ClickHouseDataType.Array) { SerializerUtils.writeNonNull(out); + } else if (dataType == ClickHouseDataType.Dynamic) { + // do nothing } else { throw new IllegalArgumentException(String.format("An attempt to write null into not nullable column '%s'", column)); } 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 a90f80af2..656c53391 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 @@ -242,9 +242,9 @@ private static T convertDateTime(ZonedDateTime value, Class typeHint) { if (typeHint == null) { return (T) value; } - if (typeHint.isAssignableFrom(LocalDateTime.class)) { + if (LocalDateTime.class.isAssignableFrom(typeHint)) { return (T) value.toLocalDateTime(); - } else if (typeHint.isAssignableFrom(LocalDate.class)) { + } else if (LocalDate.class.isAssignableFrom(typeHint)) { return (T) value.toLocalDate(); } @@ -255,7 +255,10 @@ private static T convertArray(ArrayValue value, Class typeHint) { if (typeHint == null) { return (T) value; } - if (typeHint.isAssignableFrom(List.class)) { + if (typeHint == Object.class) { + return (T) value.asList(); + } + if (List.class.isAssignableFrom(typeHint)) { return (T) value.asList(); } if (typeHint.isArray()) { @@ -918,8 +921,10 @@ public static ZonedDateTime readDateTime64(InputStream input, byte[] buff, int s } } - return LocalDateTime.ofInstant(Instant.ofEpochSecond(value, nanoSeconds), tz.toZoneId()) - .atZone(tz.toZoneId()); +// Instant.ofEpochSecond() + return Instant.ofEpochSecond(value, nanoSeconds).atZone(tz.toZoneId()); +// return LocalDateTime.ofInstant(Instant.ofEpochSecond(value, nanoSeconds), tz.toZoneId()) +// .atZone(tz.toZoneId()); } /** @@ -1025,6 +1030,16 @@ private ClickHouseColumn readDynamicData() throws IOException { throw new ClientException("Unsupported interval kind: " + intervalKind); } return ClickHouseColumn.of("v", type, false, 0, 0); + } else if (tag == ClickHouseDataType.DateTime32.getBinTag()) { + byte scale = readByte(); + return ClickHouseColumn.of("v", "DateTime32(" + scale + ")"); + } else if (tag == ClickHouseDataType.DateTime64.getBinTag() - 1) { // without timezone + byte scale = readByte(); + return ClickHouseColumn.of("v", "DateTime64(" + scale +")"); + } else if (tag == ClickHouseDataType.DateTime64.getBinTag()) { + byte scale = readByte(); + String timezone = readString(input); + return ClickHouseColumn.of("v", "DateTime64(" + scale + (timezone.isEmpty() ? "" : ", " + timezone) +")"); } else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) { String typeName = readString(input); return ClickHouseColumn.of("v", typeName); 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 deafc66f0..24b5f7e4c 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 @@ -115,12 +115,11 @@ private static Map, ClickHouseColumn> getPredefinedTypeColumnsMap() { map.put(BigInteger.class, ClickHouseColumn.of("v", "Int256")); map.put(Float.class, ClickHouseColumn.of("v", "Float32")); map.put(Double.class, ClickHouseColumn.of("v", "Float64")); - map.put(LocalDate.class, ClickHouseColumn.of("v", "Date")); map.put(UUID.class, ClickHouseColumn.of("v", "UUID")); map.put(Inet4Address.class, ClickHouseColumn.of("v", "IPv4")); map.put(Inet6Address.class, ClickHouseColumn.of("v", "IPv6")); map.put(String.class, ClickHouseColumn.of("v", "String")); - map.put(LocalDateTime.class, ClickHouseColumn.of("v", "DateTime")); + map.put(LocalDate.class, ClickHouseColumn.of("v", "Date32")); map.put(ClickHouseGeoPointValue.class, ClickHouseColumn.of("v", "Point")); map.put(ClickHouseGeoRingValue.class, ClickHouseColumn.of("v", "Ring")); @@ -162,7 +161,9 @@ public static ClickHouseColumn valueToColumnForDynamicType(Object value) { ClickHouseColumn column; if (value instanceof ZonedDateTime) { ZonedDateTime dt = (ZonedDateTime) value; - column = ClickHouseColumn.of("v", "DateTime(" + dt.getZone().getId() + ")"); + column = ClickHouseColumn.of("v", "DateTime64(9, " + dt.getZone().getId() + ")"); + } else if (value instanceof LocalDateTime) { + column = ClickHouseColumn.of("v", "DateTime64(9, " + ZoneId.systemDefault().getId() + ")"); } else if (value instanceof BigDecimal) { BigDecimal d = (BigDecimal) value; String decType; @@ -347,6 +348,9 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ BinaryStreamUtils.writeString(stream, typeColumn.getTimeZoneOrDefault(TimeZone.getDefault()).getID()); break; case DateTime64: + stream.write(binTag); + BinaryStreamUtils.writeUnsignedInt8(stream, typeColumn.getScale()); + BinaryStreamUtils.writeString(stream, typeColumn.getTimeZoneOrDefault(TimeZone.getDefault()).getID()); break; case Array: stream.write(binTag); @@ -560,6 +564,9 @@ private static void serializePrimitiveData(OutputStream stream, Object value, Cl case IntervalYear: BinaryStreamUtils.writeUnsignedInt64(stream, convertToLong(value)); break; + case Nothing: + // no value is expected to be written. Used mainly for Dynamic when NULL + break; default: throw new UnsupportedOperationException("Unsupported data type: " + column.getDataType()); } @@ -788,7 +795,7 @@ public static POJOSetter compilePOJOSetter(Method setterMethod, ClickHouseColumn Type.getType(Class.class)), false); - if (targetType.isAssignableFrom(List.class) && column.getDataType() == ClickHouseDataType.Tuple) { + if (List.class.isAssignableFrom(targetType) && column.getDataType() == ClickHouseDataType.Tuple) { mv.visitTypeInsn(CHECKCAST, Type.getInternalName(Object[].class)); mv.visitMethodInsn(INVOKESTATIC, Type.getInternalName(Arrays.class), diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java index 147c2d480..b949696c8 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java @@ -427,7 +427,7 @@ public ClassicHttpResponse executeRequest(ClickHouseNode server, Map { for (byte[] row : compressedData) { -// if (algo.) BinaryStreamUtils.writeVarInt(out, row.length); out.write(row); } @@ -639,8 +640,49 @@ public static Object[][] testAppCompressionDataProvider() { }; } + @Test(groups = { "integration" }, enabled = true) + public void testPOJOWithDynamicType() throws Exception { + final String tableName = "pojo_dynamic_type_test"; + final String createSQL = PojoWithDynamic.getTableDef(tableName); + + initTable(tableName, createSQL, (CommandSettings) + new CommandSettings().serverSetting("allow_experimental_dynamic_type", "1")); + + TableSchema tableSchema = client.getTableSchema(tableName); + client.register(PojoWithDynamic.class, tableSchema); + + List data = new ArrayList<>(); + data.add(new PojoWithDynamic(1, "test_string", null)); + data.add(new PojoWithDynamic(2, 10000L, Arrays.asList(1, 2, 3))); + data.add(new PojoWithDynamic(3, 10000L, LocalDateTime.now())); + data.add(new PojoWithDynamic(4, 10000L, ZonedDateTime.now(ZoneId.of("America/Chicago")))); + data.add(new PojoWithDynamic(5, 10000L, LocalDate.now())); + try (InsertResponse response = client.insert(tableName, data, settings) + .get(EXECUTE_CMD_TIMEOUT, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getWrittenRows(), data.size()); + } + + List items = + client.queryAll("SELECT * FROM " + tableName, PojoWithDynamic.class, tableSchema); + + int i = 0; + for (PojoWithDynamic item : items) { + if (item.rowId == 3) { + assertEquals(((ZonedDateTime) item.getNullableAny()).toLocalDateTime(), data.get(i++).getNullableAny()); + } else if (item.rowId == 5) { + assertEquals(((ZonedDateTime) item.getNullableAny()).toLocalDate(), data.get(i++).getNullableAny()); + } else { + assertEquals(item, data.get(i++)); + } + } + } + protected void initTable(String tableName, String createTableSQL) throws Exception { - client.execute("DROP TABLE IF EXISTS " + tableName).get(EXECUTE_CMD_TIMEOUT, TimeUnit.SECONDS); - client.execute(createTableSQL).get(EXECUTE_CMD_TIMEOUT, TimeUnit.SECONDS); + initTable(tableName, createTableSQL, new CommandSettings()); + } + + protected void initTable(String tableName, String createTableSQL, CommandSettings settings) throws Exception { + client.execute("DROP TABLE IF EXISTS " + tableName, settings).get(EXECUTE_CMD_TIMEOUT, TimeUnit.SECONDS); + client.execute(createTableSQL, settings).get(EXECUTE_CMD_TIMEOUT, TimeUnit.SECONDS); } } diff --git a/client-v2/src/test/java/com/clickhouse/client/insert/PojoWithDynamic.java b/client-v2/src/test/java/com/clickhouse/client/insert/PojoWithDynamic.java new file mode 100644 index 000000000..78ff6331a --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/insert/PojoWithDynamic.java @@ -0,0 +1,29 @@ +package com.clickhouse.client.insert; + +import com.clickhouse.client.datatypes.DataTypeTests; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.NoArgsConstructor; +import lombok.ToString; + +@Data +@AllArgsConstructor +@NoArgsConstructor +@EqualsAndHashCode +@ToString +public class PojoWithDynamic { + + int rowId; + + Object any; + + Object nullableAny; + + public static String getTableDef(String tableName) { + return DataTypeTests.tableDefinition(tableName, + "rowId Int32", + "any Dynamic", + "nullableAny Dynamic"); + } +} 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 4dd6d3cce..9dce8f186 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 @@ -84,7 +84,6 @@ import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; -import java.util.regex.Matcher; import java.util.stream.BaseStream; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -2091,7 +2090,7 @@ public void testGetDynamicValue() throws Exception { } @Test(groups = {"integration"}) - public void testGetJSON() throws Exception { + public void testBinaryEncodedJSON() throws Exception { if (isVersionMatch("(,24.8]")) { return; } @@ -2109,7 +2108,6 @@ public void testGetJSON() throws Exception { return a.replaceAll("'", "\""); }), 1); - System.out.println(dataset); ObjectMapper jackson = new ObjectMapper(); try (QueryResponse response = client.query("SELECT * FROM " + table).get()) { ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response); From e969608b7c7f63acc8c654f4fb5227f4a4202763 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 12 Feb 2025 20:16:38 -0800 Subject: [PATCH 14/21] fixed some tests --- .github/workflows/build.yml | 4 ++-- .../com/clickhouse/client/datatypes/DataTypeTests.java | 2 +- .../java/com/clickhouse/client/insert/InsertTests.java | 8 ++++++++ 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index d198e2133..cb6910475 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -268,7 +268,7 @@ jobs: needs: compile strategy: matrix: - clickhouse: ["23.8", "24.3", "24.6", "latest"] + clickhouse: ["23.8", "24.3", "24.8", "latest"] # here http, http_client and apache_http_client represent different value of http_connection_provider # protocol: ["http", "http_client", "apache_http_client"] protocol: ["apache_http_client"] @@ -321,7 +321,7 @@ jobs: uses: actions/upload-artifact@v4 if: failure() with: - name: result ${{ github.job }} + name: result ${{ github.job }}_${{ matrix.project }}_${{ matrix.clickhouse }} path: | **/target/failsafe-reports **/target/surefire-reports 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 760d5945e..c6c3ba9a5 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 @@ -608,7 +608,7 @@ public static String tableDefinition(String table, String... columns) { return sb.toString(); } - public boolean isVersionMatch(String versionExpression) { + private boolean isVersionMatch(String versionExpression) { List serverVersion = client.queryAll("SELECT version()"); return ClickHouseVersion.of(serverVersion.get(0).getString(1)).check(versionExpression); } diff --git a/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java b/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java index 429adf7b2..285b65cdb 100644 --- a/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java @@ -642,6 +642,9 @@ public static Object[][] testAppCompressionDataProvider() { @Test(groups = { "integration" }, enabled = true) public void testPOJOWithDynamicType() throws Exception { + if (isVersionMatch("(,24.8]")) { + return; + } final String tableName = "pojo_dynamic_type_test"; final String createSQL = PojoWithDynamic.getTableDef(tableName); @@ -685,4 +688,9 @@ protected void initTable(String tableName, String createTableSQL, CommandSetting client.execute("DROP TABLE IF EXISTS " + tableName, settings).get(EXECUTE_CMD_TIMEOUT, TimeUnit.SECONDS); client.execute(createTableSQL, settings).get(EXECUTE_CMD_TIMEOUT, TimeUnit.SECONDS); } + + private boolean isVersionMatch(String versionExpression) { + List serverVersion = client.queryAll("SELECT version()"); + return ClickHouseVersion.of(serverVersion.get(0).getString(1)).check(versionExpression); + } } From 7741b2f318e554ef2399e297fec2c9c8f4f6eb4a Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 12 Feb 2025 21:21:54 -0800 Subject: [PATCH 15/21] fixed more tests --- .../java/com/clickhouse/data/ClickHouseEnum.java | 10 ++++++++++ .../data_formats/internal/BinaryStreamReader.java | 12 ++++++++---- .../com/clickhouse/client/insert/InsertTests.java | 2 +- .../test/java/com/clickhouse/jdbc/DataTypeTests.java | 8 ++++++++ 4 files changed, 27 insertions(+), 5 deletions(-) diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseEnum.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseEnum.java index 1bf3ca2a2..877e1d203 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseEnum.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseEnum.java @@ -93,6 +93,16 @@ public String name(int value) { throw new IllegalArgumentException("Unknown enum value: " + value); } + public String nameNullable(int value) { + for (int i = 0; i < size; i++) { + if (values[i] == value) { + return names[i]; + } + } + + return null; + } + public int value(String name) { for (int i = 0; i < size; i++) { if (names[i].equals(name)) { 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 656c53391..8480e71e1 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 @@ -158,12 +158,16 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce return (T) Double.valueOf(readDoubleLE()); case Bool: return (T) Boolean.valueOf(readByteOrEOF(input) == 1); - case Enum8: + case Enum8: { byte enum8Val = (byte) readUnsignedByte(); - return (T) new EnumValue(actualColumn.getEnumConstants().name(enum8Val), enum8Val); - case Enum16: + String name = actualColumn.getEnumConstants().name(enum8Val); + return (T) new EnumValue(name == null ? "" : name, enum8Val); + } + case Enum16: { short enum16Val = (short) readUnsignedShortLE(); - return (T) new EnumValue(actualColumn.getEnumConstants().name(enum16Val), enum16Val); + String name = actualColumn.getEnumConstants().name(enum16Val); + return (T) new EnumValue(name == null ? "" : name, enum16Val); + } case Date: return convertDateTime(readDate(timezone), typeHint); case Date32: diff --git a/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java b/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java index 285b65cdb..c306ac75d 100644 --- a/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java @@ -427,7 +427,7 @@ public static Object[] logCommentDataProvider() { }; } - @Test + @Test(enabled = false) public void testWriter() throws Exception { String tableName = "very_long_table_name_with_uuid_" + UUID.randomUUID().toString().replace('-', '_'); String tableCreate = "CREATE TABLE \"" + tableName + "\" " + diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java index be11457b5..2ee34ebbc 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java @@ -823,6 +823,10 @@ public void testGeometricTypesSimpleStatement() throws SQLException { @Test(groups = { "integration" }) public void testDynamicTypesSimpleStatement() throws SQLException { + if (earlierThan(24, 8)) {//Min version is 24.4 + return; + } + Properties properties = new Properties(); properties.setProperty(ClientConfigProperties.serverSetting("allow_experimental_dynamic_type"), "1"); runQuery("CREATE TABLE test_dynamic (order Int8, " @@ -935,6 +939,10 @@ public void testTypeConversions() throws Exception { @Test(groups = { "integration" }) public void testVariantTypesSimpleStatement() throws SQLException { + if (earlierThan(24, 8)) {//Min version is 24.4 + return; + } + Properties properties = new Properties(); properties.setProperty(ClientConfigProperties.serverSetting("allow_experimental_variant_type"), "1"); runQuery("CREATE TABLE test_variant (order Int8, " From b93815a73401cf6ed9304ff80879572dd6c5fbe0 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 12 Feb 2025 21:45:27 -0800 Subject: [PATCH 16/21] fixed proper settings for cloud --- .../data_formats/internal/BinaryStreamReader.java | 4 ++-- .../java/com/clickhouse/client/query/QueryTests.java | 2 +- .../test/java/com/clickhouse/jdbc/DataTypeTests.java | 12 ++++++++---- 3 files changed, 11 insertions(+), 7 deletions(-) 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 8480e71e1..75225b873 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 @@ -160,12 +160,12 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce return (T) Boolean.valueOf(readByteOrEOF(input) == 1); case Enum8: { byte enum8Val = (byte) readUnsignedByte(); - String name = actualColumn.getEnumConstants().name(enum8Val); + String name = actualColumn.getEnumConstants().nameNullable(enum8Val); return (T) new EnumValue(name == null ? "" : name, enum8Val); } case Enum16: { short enum16Val = (short) readUnsignedShortLE(); - String name = actualColumn.getEnumConstants().name(enum16Val); + String name = actualColumn.getEnumConstants().nameNullable(enum16Val); return (T) new EnumValue(name == null ? "" : name, enum16Val); } case Date: 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 9dce8f186..17a6ec45d 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 @@ -1418,7 +1418,7 @@ private List> prepareDataSet(String table, List colu // Create table CommandSettings settings = new CommandSettings(); - if (isVersionMatch("[24.8,)")) { + if (isVersionMatch("[24.8,)") && !isCloud()) { settings.serverSetting("allow_experimental_dynamic_type", "1") .serverSetting("allow_experimental_json_type", "1"); } diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java index 2ee34ebbc..29b66508a 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java @@ -828,11 +828,13 @@ public void testDynamicTypesSimpleStatement() throws SQLException { } Properties properties = new Properties(); - properties.setProperty(ClientConfigProperties.serverSetting("allow_experimental_dynamic_type"), "1"); - runQuery("CREATE TABLE test_dynamic (order Int8, " + if (!isCloud()) { + properties.setProperty(ClientConfigProperties.serverSetting("allow_experimental_dynamic_type"), "1"); + } + assertTrue(runQuery("CREATE TABLE test_dynamic (order Int8, " + "dynamic Dynamic" + ") ENGINE = MergeTree ORDER BY ()", - properties); + properties), "Failed to create table"); // Insert random (valid) values long seed = System.currentTimeMillis(); @@ -944,7 +946,9 @@ public void testVariantTypesSimpleStatement() throws SQLException { } Properties properties = new Properties(); - properties.setProperty(ClientConfigProperties.serverSetting("allow_experimental_variant_type"), "1"); + if (!isCloud()) { + properties.setProperty(ClientConfigProperties.serverSetting("allow_experimental_variant_type"), "1"); + } runQuery("CREATE TABLE test_variant (order Int8, " + "v Variant(String, Int32)" + ") ENGINE = MergeTree ORDER BY ()", From 1b81ce5a61d1c3a876753d203e77611716c60046 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 12 Feb 2025 22:40:26 -0800 Subject: [PATCH 17/21] one more test fix --- jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java index 29b66508a..d22a5796d 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java @@ -831,10 +831,10 @@ public void testDynamicTypesSimpleStatement() throws SQLException { if (!isCloud()) { properties.setProperty(ClientConfigProperties.serverSetting("allow_experimental_dynamic_type"), "1"); } - assertTrue(runQuery("CREATE TABLE test_dynamic (order Int8, " + runQuery("CREATE TABLE test_dynamic (order Int8, " + "dynamic Dynamic" + ") ENGINE = MergeTree ORDER BY ()", - properties), "Failed to create table"); + properties); // Insert random (valid) values long seed = System.currentTimeMillis(); From eb68c763162e8b7de18d3a50a16a0cc538b52937 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 13 Feb 2025 12:10:22 -0800 Subject: [PATCH 18/21] Fixed missing isSecure for cloud tests --- .../java/com/clickhouse/client/ClickHouseServerForTest.java | 1 + .../java/com/clickhouse/client/datatypes/DataTypeTests.java | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseServerForTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseServerForTest.java index f2d1ae055..a86325c65 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseServerForTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseServerForTest.java @@ -358,6 +358,7 @@ public static void beforeSuite() { @AfterSuite(groups = {"integration"}) public static void afterSuite() { if (clickhouseContainer != null) { +// clickhouseContainer.copyFileFromContainer("/var/log/clickhouse-server/clickhouse-server.log", "server-container.log"); clickhouseContainer.stop(); } 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 c6c3ba9a5..c33c4e8bb 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 @@ -58,7 +58,7 @@ public DataTypeTests() { public void setUp() throws IOException { ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); client = new Client.Builder() - .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), false) + .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), isCloud()) .setUsername("default") .setPassword(ClickHouseServerForTest.getPassword()) .compressClientRequest(useClientCompression) @@ -179,9 +179,9 @@ public void testVariantWithSimpleDataTypes() throws Exception { continue dataTypesLoop; } - b.append(")) Engine = MergeTree ORDER BY ()"); + b.append(")) Engine = MergeTree ORDER BY () SETTINGS enable_variant_type=1"); - client.execute(b.toString(), (CommandSettings) new CommandSettings().serverSetting("enable_variant_type", "1")); + client.execute(b.toString()); client.register(DTOForVariantPrimitivesTests.class, client.getTableSchema(table)); Object value = null; From 296d6760ff3c34b5a13acaeaa7da8e6c0fd896cf Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 13 Feb 2025 13:16:58 -0800 Subject: [PATCH 19/21] addressed some issues --- .github/workflows/build.yml | 6 +++--- .../java/com/clickhouse/client/datatypes/DataTypeTests.java | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index cb6910475..3a44b2118 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -145,7 +145,7 @@ jobs: matrix: # most recent LTS releases as well as latest stable builds # https://github.com/ClickHouse/ClickHouse/pulls?q=is%3Aopen+is%3Apr+label%3Arelease - clickhouse: ["23.8", "24.3", "24.8", "latest"] + clickhouse: ["24.3", "24.8", "latest"] project: ["clickhouse-http-client", "client-v2"] fail-fast: false timeout-minutes: 15 @@ -268,7 +268,7 @@ jobs: needs: compile strategy: matrix: - clickhouse: ["23.8", "24.3", "24.8", "latest"] + clickhouse: ["24.3", "24.8", "latest"] # here http, http_client and apache_http_client represent different value of http_connection_provider # protocol: ["http", "http_client", "apache_http_client"] protocol: ["apache_http_client"] @@ -331,7 +331,7 @@ jobs: needs: compile strategy: matrix: - clickhouse: ["23.8", "24.3", "24.8", "latest"] + clickhouse: ["24.3", "24.8", "latest"] # grpc is not fully supported, and http_client and apache_http_client do not work in CI environment(due to limited threads?) protocol: ["http"] r2dbc: ["1.0.0.RELEASE", "0.9.1.RELEASE"] 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 c33c4e8bb..7317b25c7 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 @@ -179,7 +179,7 @@ public void testVariantWithSimpleDataTypes() throws Exception { continue dataTypesLoop; } - b.append(")) Engine = MergeTree ORDER BY () SETTINGS enable_variant_type=1"); + b.append(")) Engine = MergeTree ORDER BY () SETTINGS allow_experimental_dynamic_type=1"); client.execute(b.toString()); client.register(DTOForVariantPrimitivesTests.class, client.getTableSchema(table)); From 17b8770f8a31341a5644b6d74f805e8f381529ee Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 13 Feb 2025 19:50:00 -0800 Subject: [PATCH 20/21] added interval serialization/deserialization --- .../clickhouse/data/ClickHouseDataType.java | 45 +++++++------- .../ClickHouseBinaryFormatReader.java | 5 ++ .../internal/AbstractBinaryFormatReader.java | 47 +++++--------- .../internal/BinaryReaderBackedRecord.java | 11 ++++ .../internal/BinaryStreamReader.java | 40 ++++++++++-- .../internal/MapBackedRecord.java | 44 ++++--------- .../internal/SerializerUtils.java | 58 ++++++++++++++++- .../client/api/query/GenericRecord.java | 15 +++++ .../internal/SerializerUtilsTests.java | 5 ++ .../client/datatypes/DataTypeTests.java | 32 ++++++++-- .../datatypes/DataTypesTestingPOJO.java | 62 ++++++++++--------- .../com/clickhouse/jdbc/DataTypeTests.java | 4 +- 12 files changed, 238 insertions(+), 130 deletions(-) create mode 100644 client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/SerializerUtilsTests.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 86e196bc2..a8763cee0 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -1,6 +1,5 @@ package com.clickhouse.data; -import java.lang.reflect.Array; import java.math.BigDecimal; import java.math.BigInteger; import java.net.Inet4Address; @@ -8,12 +7,11 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.ZonedDateTime; -import java.util.ArrayList; +import java.time.temporal.ChronoUnit; +import java.time.temporal.TemporalUnit; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.Comparator; -import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; @@ -247,36 +245,39 @@ private static Set> setOf(Class... args) { public static final byte TUPLE_WITH_NAMES_BIN_TAG = 0x20; - public enum IntervalKindBinTag { - Nanosecond(IntervalNanosecond, 0x00), - Microsecond(IntervalMicrosecond, 0x01), + public enum IntervalKind { + Nanosecond(IntervalNanosecond, ChronoUnit.NANOS, 0x00), + Microsecond(IntervalMicrosecond, ChronoUnit.MICROS, 0x01), - Millisecond(IntervalMillisecond, 0x02), + Millisecond(IntervalMillisecond, ChronoUnit.MILLIS, 0x02), - Second(IntervalSecond, 0x03), + Second(IntervalSecond, ChronoUnit.SECONDS, 0x03), - Minute(IntervalMinute, 0x04), + Minute(IntervalMinute, ChronoUnit.MINUTES, 0x04), - Hour(IntervalHour, 0x05), + Hour(IntervalHour, ChronoUnit.HOURS, 0x05), - Day(IntervalDay, 0x06), + Day(IntervalDay, ChronoUnit.DAYS, 0x06), - Week(IntervalWeek, 0x07), + Week(IntervalWeek, ChronoUnit.WEEKS, 0x07), - Month(IntervalMonth, 0x08), + Month(IntervalMonth, ChronoUnit.MONTHS, 0x08), - Quarter(IntervalQuarter, 0x09), + Quarter(IntervalQuarter, null, 0x09), - Year(IntervalYear, 0x1A) // why 1A ? + Year(IntervalYear, ChronoUnit.YEARS, 0x1A) // why 1A ? ; private ClickHouseDataType intervalType; + private TemporalUnit temporalUnit; + byte tag; - IntervalKindBinTag(ClickHouseDataType clickHouseDataType, int tag) { + IntervalKind(ClickHouseDataType clickHouseDataType, TemporalUnit temporalUnit, int tag) { this.intervalType = clickHouseDataType; this.tag = (byte) tag; + this.temporalUnit = temporalUnit; } public ClickHouseDataType getIntervalType() { @@ -286,6 +287,8 @@ public ClickHouseDataType getIntervalType() { public byte getTag() { return tag; } + + public TemporalUnit getTemporalUnit() { return temporalUnit; } } @@ -303,7 +306,7 @@ public byte getTag() { public static final Map intervalKind2Type; - public static final Map intervalType2Kind; + public static final Map intervalType2Kind; static { Set set = new TreeSet<>(); @@ -341,10 +344,10 @@ public byte getTag() { binTag2Type = Collections.unmodifiableMap(tmpbinTag2Type); Map tmpIntervalKind2Type = new HashMap<>(); - Map tmpIntervalType2Kind = new HashMap<>(); - for (IntervalKindBinTag kind : IntervalKindBinTag.values()) { + Map tmpIntervalType2Kind = new HashMap<>(); + for (IntervalKind kind : IntervalKind.values()) { tmpIntervalKind2Type.put(kind.getTag(), kind.getIntervalType()); - tmpIntervalType2Kind.put(kind.getIntervalType(), kind.tag); + tmpIntervalType2Kind.put(kind.getIntervalType(), kind); } intervalKind2Type = Collections.unmodifiableMap(tmpIntervalKind2Type); intervalType2Kind = Collections.unmodifiableMap(tmpIntervalType2Kind); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java index b0edfac56..ec08e4ea3 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java @@ -12,6 +12,7 @@ import java.net.Inet4Address; import java.net.Inet6Address; import java.time.*; +import java.time.temporal.TemporalAmount; import java.util.List; import java.util.Map; import java.util.UUID; @@ -540,4 +541,8 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable { ClickHouseBitmap getClickHouseBitmap(String colName); ClickHouseBitmap getClickHouseBitmap(int index); + + TemporalAmount getTemporalAmount(int index); + + TemporalAmount getTemporalAmount(String colName); } 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 4eed17f27..77ca69e59 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 @@ -30,6 +30,7 @@ import java.time.*; import java.time.format.DateTimeFormatter; import java.time.temporal.ChronoUnit; +import java.time.temporal.TemporalAmount; import java.util.*; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; @@ -414,38 +415,13 @@ public ZonedDateTime getZonedDateTime(String colName) { @Override public Duration getDuration(String colName) { - int colIndex = schema.nameToIndex(colName); - ClickHouseColumn column = schema.getColumns().get(colIndex); - BigInteger value = readValue(colName); - try { - switch (column.getDataType()) { - case IntervalYear: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.YEARS); - case IntervalQuarter: - return Duration.of(value.longValue() * 3, java.time.temporal.ChronoUnit.MONTHS); - case IntervalMonth: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.MONTHS); - case IntervalWeek: - return Duration.of(value.longValue(), ChronoUnit.WEEKS); - case IntervalDay: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.DAYS); - case IntervalHour: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.HOURS); - case IntervalMinute: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.MINUTES); - case IntervalSecond: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.SECONDS); - case IntervalMicrosecond: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.MICROS); - case IntervalMillisecond: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.MILLIS); - case IntervalNanosecond: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.NANOS); - } - } catch (ArithmeticException e) { - throw new ClientException("Stored value is bigger then Long.MAX_VALUE and it cannot be converted to Duration without information loss", e); - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Duration"); + TemporalAmount temporalAmount = getTemporalAmount(colName); + return Duration.from(temporalAmount); + } + + @Override + public TemporalAmount getTemporalAmount(String colName) { + return readValue(colName); } @Override @@ -606,7 +582,12 @@ public ZonedDateTime getZonedDateTime(int index) { @Override public Duration getDuration(int index) { - return readValue(index); + return getDuration(schema.columnIndexToName(index)); + } + + @Override + public TemporalAmount getTemporalAmount(int index) { + return getTemporalAmount(schema.columnIndexToName(index)); } @Override diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java index abf26015e..4e6a9ce12 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java @@ -11,6 +11,7 @@ import java.net.Inet4Address; import java.net.Inet6Address; import java.time.*; +import java.time.temporal.TemporalAmount; import java.util.List; import java.util.Map; import java.util.UUID; @@ -89,6 +90,11 @@ public Duration getDuration(String colName) { return reader.getDuration(colName); } + @Override + public TemporalAmount getTemporalAmount(String colName) { + return reader.getTemporalAmount(colName); + } + @Override public Inet4Address getInet4Address(String colName) { return reader.getInet4Address(colName); @@ -234,6 +240,11 @@ public Duration getDuration(int index) { return reader.getDuration(index); } + @Override + public TemporalAmount getTemporalAmount(int index) { + return reader.getTemporalAmount(index); + } + @Override public Inet4Address getInet4Address(int index) { return reader.getInet4Address(index); 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 75225b873..25f9a9b2c 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 @@ -17,10 +17,13 @@ import java.net.Inet4Address; import java.net.Inet6Address; import java.nio.charset.StandardCharsets; +import java.time.Duration; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; +import java.time.Period; import java.time.ZonedDateTime; +import java.time.temporal.TemporalAmount; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -189,7 +192,7 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce case IntervalMicrosecond: case IntervalMillisecond: case IntervalNanosecond: - return (T) readBigIntegerLE(8, true); + return (T) readIntervalValue(dataType, input); case IPv4: // https://clickhouse.com/docs/en/sql-reference/data-types/ipv4 return (T) Inet4Address.getByAddress(readNBytesLE(input, 4)); @@ -218,7 +221,6 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce return convertArray(readArray(actualColumn), typeHint); case Map: return (T) readMap(actualColumn); -// case Nested: case Tuple: return (T) readTuple(actualColumn); case Nothing: @@ -242,6 +244,37 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce } } + private TemporalAmount readIntervalValue(ClickHouseDataType dataType, InputStream input) throws IOException { + BigInteger v = readBigIntegerLE(8, true); + + switch (dataType) { + case IntervalYear: + return Period.ofYears(v.intValue()); + case IntervalQuarter: + return Period.ofMonths(4 * v.intValue()); + case IntervalMonth: + return Period.ofMonths(v.intValue()); + case IntervalWeek: + return Period.ofWeeks(v.intValue()); + case IntervalDay: + return Period.ofDays(v.intValue()); + case IntervalHour: + return Duration.ofHours(v.longValue()); + case IntervalMinute: + return Duration.ofMinutes(v.longValue()); + case IntervalSecond: + return Duration.ofSeconds(v.longValue()); + case IntervalMicrosecond: + return Duration.ofNanos(1000 * v.longValue()); + case IntervalMillisecond: + return Duration.ofMillis(v.longValue()); + case IntervalNanosecond: + return Duration.ofNanos(v.longValue()); + default: + throw new ClientException("Unsupported interval type: " + dataType); + } + } + private static T convertDateTime(ZonedDateTime value, Class typeHint) { if (typeHint == null) { return (T) value; @@ -925,10 +958,7 @@ public static ZonedDateTime readDateTime64(InputStream input, byte[] buff, int s } } -// Instant.ofEpochSecond() return Instant.ofEpochSecond(value, nanoSeconds).atZone(tz.toZoneId()); -// return LocalDateTime.ofInstant(Instant.ofEpochSecond(value, nanoSeconds), tz.toZoneId()) -// .atZone(tz.toZoneId()); } /** 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 0542fcf7c..813584994 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 @@ -5,6 +5,7 @@ import com.clickhouse.client.api.query.GenericRecord; import com.clickhouse.client.api.query.NullValueException; import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.ClickHouseDataType; import com.clickhouse.data.value.*; import java.math.BigDecimal; @@ -13,6 +14,7 @@ import java.net.Inet6Address; import java.time.*; import java.time.temporal.ChronoUnit; +import java.time.temporal.TemporalAmount; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -150,37 +152,12 @@ public ZonedDateTime getZonedDateTime(String colName) { @Override public Duration getDuration(String colName) { - ClickHouseColumn column = schema.getColumnByName(colName); - BigInteger value = readValue(colName); - try { - switch (column.getDataType()) { - case IntervalYear: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.YEARS); - case IntervalQuarter: - return Duration.of(value.longValue() * 3, java.time.temporal.ChronoUnit.MONTHS); - case IntervalMonth: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.MONTHS); - case IntervalWeek: - return Duration.of(value.longValue(), ChronoUnit.WEEKS); - case IntervalDay: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.DAYS); - case IntervalHour: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.HOURS); - case IntervalMinute: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.MINUTES); - case IntervalSecond: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.SECONDS); - case IntervalMicrosecond: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.MICROS); - case IntervalMillisecond: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.MILLIS); - case IntervalNanosecond: - return Duration.of(value.longValue(), java.time.temporal.ChronoUnit.NANOS); - } - } catch (ArithmeticException e) { - throw new ClientException("Stored value is bigger then Long.MAX_VALUE and it cannot be converted to Duration without information loss", e); - } - throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Duration"); + return readValue(colName); + } + + @Override + public TemporalAmount getTemporalAmount(String colName) { + return readValue(colName); } @Override @@ -339,6 +316,11 @@ public Duration getDuration(int index) { return readValue(index); } + @Override + public TemporalAmount getTemporalAmount(int index) { + return readValue(index); + } + @Override public Inet4Address getInet4Address(int index) { return readValue(index); 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 24b5f7e4c..83f618aa1 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 @@ -30,6 +30,7 @@ import java.net.Inet6Address; import java.sql.Timestamp; import java.time.*; +import java.time.temporal.TemporalUnit; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -120,6 +121,8 @@ private static Map, ClickHouseColumn> getPredefinedTypeColumnsMap() { map.put(Inet6Address.class, ClickHouseColumn.of("v", "IPv6")); map.put(String.class, ClickHouseColumn.of("v", "String")); map.put(LocalDate.class, ClickHouseColumn.of("v", "Date32")); + map.put(Duration.class, ClickHouseColumn.of("v", "IntervalNanosecond")); + map.put(Period.class, ClickHouseColumn.of("v", "IntervalDay")); map.put(ClickHouseGeoPointValue.class, ClickHouseColumn.of("v", "Point")); map.put(ClickHouseGeoRingValue.class, ClickHouseColumn.of("v", "Ring")); @@ -337,7 +340,7 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ case IntervalQuarter: case IntervalYear: stream.write(binTag); - Byte kindTag = ClickHouseDataType.intervalType2Kind.get(dt); + Byte kindTag = ClickHouseDataType.intervalType2Kind.get(dt).getTag(); if (kindTag == null) { throw new ClientException("BUG! No Interval Mapping to a kind tag"); } @@ -562,7 +565,7 @@ private static void serializePrimitiveData(OutputStream stream, Object value, Cl case IntervalMonth: case IntervalQuarter: case IntervalYear: - BinaryStreamUtils.writeUnsignedInt64(stream, convertToLong(value)); + serializeInterval(stream, column, value); break; case Nothing: // no value is expected to be written. Used mainly for Dynamic when NULL @@ -572,6 +575,57 @@ private static void serializePrimitiveData(OutputStream stream, Object value, Cl } } + private static void serializeInterval(OutputStream stream, ClickHouseColumn column, Object value) throws IOException { + long v; + + if (value instanceof Duration) { + Duration d = (Duration) value; + switch (column.getDataType()) { + case IntervalMillisecond: + v = d.toMillis(); + break; + case IntervalNanosecond: + v = d.toNanos(); + break; + case IntervalMicrosecond: + v = d.toNanos() / 1000; + break; + case IntervalSecond: + v = d.getSeconds(); + break; + case IntervalMinute: + v = d.toMinutes(); + break; + case IntervalHour: + v = d.toHours(); + break; + case IntervalDay: + v = d.toDays(); + break; + default: + throw new UnsupportedOperationException("Cannot convert Duration to " + column.getDataType()); + } + } else if (value instanceof Period) { + Period p = (Period) value; + switch (column.getDataType()) { + case IntervalDay: + v = p.toTotalMonths() * 30 + p.getDays(); + break; + case IntervalMonth: + v = p.toTotalMonths(); + break; + case IntervalYear: + v = p.toTotalMonths() / 12; + break; + default: + throw new UnsupportedOperationException("Cannot convert Period to " + column.getDataType()); + } + } else { + throw new UnsupportedOperationException("Cannot convert " + value.getClass() + " to " + column.getDataType()); + } + BinaryStreamUtils.writeUnsignedInt64(stream, v); + } + private static void serializeEnumData(OutputStream stream, ClickHouseColumn column, Object value) throws IOException { int enumValue = -1; if (value instanceof String) { diff --git a/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java index db27d43e1..4fe4c4106 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java @@ -8,6 +8,7 @@ import java.net.Inet4Address; import java.net.Inet6Address; import java.time.*; +import java.time.temporal.TemporalAmount; import java.util.List; import java.util.Map; import java.util.UUID; @@ -131,6 +132,13 @@ public interface GenericRecord { */ Duration getDuration(String colName); + /** + * Returns the value of the specified column as a TemporalAmount. + * Period is used for dates and Duration is used for times. + * @param colName + * @return TemporalAmount in the unit of column type. + */ + TemporalAmount getTemporalAmount(String colName); /** * Returns the value of the specified column as an Inet4Address. @@ -359,6 +367,13 @@ public interface GenericRecord { */ Duration getDuration(int index); + /** + * Returns the value of the specified column as a TemporalAmount. + * Period is used for dates and Duration is used for times. + * @param index + * @return TemporalAmount in the unit of column type. + */ + TemporalAmount getTemporalAmount(int index); /** * Returns the value of the specified column as an Inet4Address. diff --git a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/SerializerUtilsTests.java b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/SerializerUtilsTests.java new file mode 100644 index 000000000..578b5f172 --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/SerializerUtilsTests.java @@ -0,0 +1,5 @@ +package com.clickhouse.client.api.data_formats.internal; + +public class SerializerUtilsTests { + +} 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 7317b25c7..ddc3447ad 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 @@ -23,8 +23,12 @@ import java.io.IOException; import java.lang.reflect.Method; import java.math.BigDecimal; +import java.time.Duration; import java.time.LocalDateTime; +import java.time.Period; import java.time.temporal.ChronoUnit; +import java.time.temporal.TemporalAmount; +import java.time.temporal.TemporalUnit; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -138,7 +142,7 @@ public static String tblCreateSQL(String table) { @Test(groups = {"integration"}) public void testVariantWithSimpleDataTypes() throws Exception { - if (isVersionMatch("(,24.8]")) { + if (isVersionMatch("(,24.8]") || isCloud()) { return; } @@ -179,7 +183,7 @@ public void testVariantWithSimpleDataTypes() throws Exception { continue dataTypesLoop; } - b.append(")) Engine = MergeTree ORDER BY () SETTINGS allow_experimental_dynamic_type=1"); + b.append(")) Engine = MergeTree ORDER BY () SETTINGS enable_variant_type=1"); client.execute(b.toString()); client.register(DTOForVariantPrimitivesTests.class, client.getTableSchema(table)); @@ -372,8 +376,7 @@ public void testVariantWithTuple() throws Exception { @Test(groups = {"integration"}) public void testDynamicWithPrimitives() throws Exception { - - if (isVersionMatch("(,24.8]")) { + if (isVersionMatch("(,24.8]") || isCloud()) { return; } @@ -461,6 +464,23 @@ public void testDynamicWithPrimitives() throws Exception { BigDecimal tmpDec = row.getBigDecimal("field").stripTrailingZeros(); strValue = tmpDec.toPlainString(); break; + case IntervalMicrosecond: + case IntervalMillisecond: + case IntervalSecond: + case IntervalMinute: + case IntervalHour: + strValue = String.valueOf(row.getTemporalAmount("field")); + break; + case IntervalDay: + case IntervalWeek: + case IntervalMonth: + case IntervalQuarter: + case IntervalYear: + strValue = String.valueOf(row.getTemporalAmount("field")); + Period period = (Period) value; + long days = (period).getDays() + Math.round((period.toTotalMonths() / 12d) * 360); + value = Period.ofDays((int) days); + break; } if (value.getClass().isPrimitive()) { Assert.assertEquals(strValue, String.valueOf(value)); @@ -548,7 +568,7 @@ public static class DTOForDynamicPrimitivesTests { } private void testDynamicWith(String withWhat, Object[] values, String[] expectedStrValues) throws Exception { - if (isVersionMatch("(,24.8]")) { + if (isVersionMatch("(,24.8]") || isCloud()) { return; } @@ -572,7 +592,7 @@ private void testDynamicWith(String withWhat, Object[] values, String[] expected } private void testVariantWith(String withWhat, String[] fields, Object[] values, String[] expectedStrValues) throws Exception { - if (isVersionMatch("(,24.8]")) { + if (isVersionMatch("(,24.8]") || isCloud()) { return; } 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 index cae20b008..24a258859 100644 --- a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java @@ -14,8 +14,10 @@ import java.net.Inet4Address; import java.net.Inet6Address; import java.net.UnknownHostException; +import java.time.Duration; import java.time.LocalDate; import java.time.LocalDateTime; +import java.time.Period; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -94,27 +96,27 @@ public class DataTypesTestingPOJO { private ClickHouseBitmap groupBitmapUint32; private ClickHouseBitmap groupBitmapUint64; - private int intervalYear; + private Period intervalYear; - private byte intervalQuarter; + private Period intervalQuarter; - private byte intervalMonth; + private Period intervalMonth; - private byte intervalWeek; + private Period intervalWeek; - private short intervalDay; + private Period intervalDay; - private byte intervalHour; + private Duration intervalHour; - private byte intervalMinute; + private Duration intervalMinute; - private byte intervalSecond; + private Duration intervalSecond; - private long intervalMillisecond; + private Duration intervalMillisecond; - private long intervalMicrosecond; + private Duration intervalMicrosecond; - private BigInteger intervalNanosecond; + private Duration intervalNanosecond; private SmallEnum smallEnum; @@ -245,18 +247,18 @@ public DataTypesTestingPOJO() { 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()); - intervalYear = random.nextInt(4000); - intervalQuarter = (byte) random.nextInt(4); - intervalMonth = (byte) random.nextInt(12); - intervalWeek = (byte) random.nextInt(52); - intervalDay = (byte) random.nextInt(30); - intervalHour = (byte) random.nextInt(24); - intervalMinute = (byte) random.nextInt(60); - intervalSecond = (byte) random.nextInt(60); - intervalMillisecond = Math.abs(random.nextLong()); - intervalMicrosecond = Math.abs(random.nextInt()); + intervalYear = Period.ofYears(random.nextInt(4000)); + intervalQuarter = Period.ofDays(random.nextInt(1000)); + intervalMonth = Period.ofMonths(random.nextInt(12)); + intervalWeek = Period.ofWeeks(random.nextInt(52)); + intervalDay = Period.ofDays(random.nextInt(30)); + intervalHour = Duration.ofHours(random.nextInt(24)); + intervalMinute = Duration.ofMinutes(random.nextInt(60)); + intervalSecond = Duration.ofSeconds(random.nextInt(60)); + intervalMillisecond = Duration.ofMillis(random.nextInt()); + intervalMicrosecond = Duration.ofNanos(random.nextInt()); - intervalNanosecond = BigInteger.valueOf(Math.abs(random.nextLong())); + intervalNanosecond = Duration.ofNanos((random.nextInt())); smallEnum = SmallEnum.valueOf("CONSTANT_" + Math.max(1, random.nextInt(SmallEnum.values().length - 1))); largeEnum = LargeEnum.valueOf("CONSTANT_" + Math.max(1, random.nextInt(LargeEnum.values().length - 1))); @@ -319,16 +321,16 @@ public static String generateTableCreateSQL(String tableName) { "innerNullableInt Nullable(Int32)), " + "groupBitmapUint32 AggregateFunction(groupBitmap, UInt32), " + "groupBitmapUint64 AggregateFunction(groupBitmap, UInt64), " + - "intervalYear IntervalYear, " + - "intervalQuarter IntervalQuarter, " + - "intervalMonth IntervalMonth, " + - "intervalWeek IntervalWeek, " + + "intervalYear IntervalDay, " + + "intervalQuarter IntervalDay, " + + "intervalMonth IntervalDay, " + + "intervalWeek IntervalDay, " + "intervalDay IntervalDay, " + "intervalHour IntervalHour, " + - "intervalMinute IntervalMinute, " + - "intervalSecond IntervalSecond, " + - "intervalMillisecond IntervalMillisecond, " + - "intervalMicrosecond IntervalMicrosecond, " + + "intervalMinute IntervalNanosecond, " + + "intervalSecond IntervalNanosecond, " + + "intervalMillisecond IntervalNanosecond, " + + "intervalMicrosecond IntervalNanosecond, " + "intervalNanosecond IntervalNanosecond " + ") ENGINE = MergeTree ORDER BY ()"; } diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java index d22a5796d..b3a9f5c5b 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java @@ -823,7 +823,7 @@ public void testGeometricTypesSimpleStatement() throws SQLException { @Test(groups = { "integration" }) public void testDynamicTypesSimpleStatement() throws SQLException { - if (earlierThan(24, 8)) {//Min version is 24.4 + if (earlierThan(24, 8) || isCloud()) { return; } @@ -941,7 +941,7 @@ public void testTypeConversions() throws Exception { @Test(groups = { "integration" }) public void testVariantTypesSimpleStatement() throws SQLException { - if (earlierThan(24, 8)) {//Min version is 24.4 + if (earlierThan(24, 8) || isCloud()) { return; } From edc87a16886fc72a7c60c4cbd5f26e42bae3d58c Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 13 Feb 2025 22:23:46 -0800 Subject: [PATCH 21/21] fixed Period serialization --- .../clickhouse/data/ClickHouseDataType.java | 26 +++++++++++-------- .../internal/BinaryStreamReader.java | 4 +-- .../internal/SerializerUtils.java | 10 +++++-- .../client/datatypes/DataTypeTests.java | 20 ++++++++++++++ .../datatypes/DataTypesTestingPOJO.java | 4 +-- 5 files changed, 47 insertions(+), 17 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 a8763cee0..885237021 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -4,8 +4,10 @@ import java.math.BigInteger; import java.net.Inet4Address; import java.net.Inet6Address; +import java.time.Duration; import java.time.LocalDate; import java.time.LocalDateTime; +import java.time.Period; import java.time.ZonedDateTime; import java.time.temporal.ChronoUnit; import java.time.temporal.TemporalUnit; @@ -212,17 +214,19 @@ static Map>> dataTypeClassMap() { 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)); - map.put(IntervalYear, setOfAllNumberClasses); - map.put(IntervalQuarter, setOfAllNumberClasses); - map.put(IntervalMonth, setOfAllNumberClasses); - map.put(IntervalWeek, setOfAllNumberClasses); - map.put(IntervalDay, setOfAllNumberClasses); - map.put(IntervalHour, setOfAllNumberClasses); - map.put(IntervalMinute, setOfAllNumberClasses); - map.put(IntervalSecond, setOfAllNumberClasses); - map.put(IntervalMillisecond, setOfAllNumberClasses); - map.put(IntervalMicrosecond, setOfAllNumberClasses); - map.put(IntervalNanosecond, setOfAllNumberClasses); + Set> dateIntervalClasses = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(Period.class, Duration.class, byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class, BigInteger.class))); + Set> timeIntervalClasses = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(Duration.class, byte.class, Byte.class, short.class, Short.class, int.class, Integer.class, long.class, Long.class, BigInteger.class))); + map.put(IntervalYear, dateIntervalClasses); + map.put(IntervalQuarter, dateIntervalClasses); + map.put(IntervalMonth, dateIntervalClasses); + map.put(IntervalWeek, dateIntervalClasses); + map.put(IntervalDay, dateIntervalClasses); + map.put(IntervalHour, timeIntervalClasses); + map.put(IntervalMinute, timeIntervalClasses); + map.put(IntervalSecond, timeIntervalClasses); + map.put(IntervalMillisecond, timeIntervalClasses); + map.put(IntervalMicrosecond, timeIntervalClasses); + map.put(IntervalNanosecond, timeIntervalClasses); return map; } 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 25f9a9b2c..334d544f8 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 @@ -251,7 +251,7 @@ private TemporalAmount readIntervalValue(ClickHouseDataType dataType, InputStrea case IntervalYear: return Period.ofYears(v.intValue()); case IntervalQuarter: - return Period.ofMonths(4 * v.intValue()); + return Period.ofMonths(3 * v.intValue()); case IntervalMonth: return Period.ofMonths(v.intValue()); case IntervalWeek: @@ -265,7 +265,7 @@ private TemporalAmount readIntervalValue(ClickHouseDataType dataType, InputStrea case IntervalSecond: return Duration.ofSeconds(v.longValue()); case IntervalMicrosecond: - return Duration.ofNanos(1000 * v.longValue()); + return Duration.ofNanos(v.longValue() * 1000); case IntervalMillisecond: return Duration.ofMillis(v.longValue()); case IntervalNanosecond: 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 83f618aa1..6a61130ed 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 @@ -611,11 +611,17 @@ private static void serializeInterval(OutputStream stream, ClickHouseColumn colu case IntervalDay: v = p.toTotalMonths() * 30 + p.getDays(); break; + case IntervalWeek: + v = (p.toTotalMonths() * 30 + p.getDays()) / 7; + break; case IntervalMonth: - v = p.toTotalMonths(); + v = p.toTotalMonths() + p.getDays() / 30; + break; + case IntervalQuarter: + v = (p.toTotalMonths() + (p.getDays() / 30)) / 3; break; case IntervalYear: - v = p.toTotalMonths() / 12; + v = (p.toTotalMonths() + (p.getDays() / 30)) / 12; break; default: throw new UnsupportedOperationException("Cannot convert Period to " + column.getDataType()); 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 ddc3447ad..9f276f7d7 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 @@ -227,6 +227,26 @@ public void testVariantWithSimpleDataTypes() throws Exception { case MultiPolygon: strValue = row.getGeoMultiPolygon("field").toString(); break; + case IntervalMicrosecond: + case IntervalMillisecond: + case IntervalSecond: + case IntervalMinute: + case IntervalHour: + strValue = String.valueOf(row.getTemporalAmount("field")); + break; + case IntervalDay: + case IntervalWeek: + case IntervalMonth: + case IntervalQuarter: + case IntervalYear: + Period period = (Period) value; + long days = (period).getDays() + Math.round((period.toTotalMonths() * 30)); + value = Period.ofDays((int) days); + period = (Period) row.getTemporalAmount("field"); + days = (period).getDays() + Math.round((period.toTotalMonths() * 30)); + strValue = Period.ofDays((int) days).toString(); + break; + } if (value.getClass().isPrimitive()) { Assert.assertEquals(strValue, String.valueOf(value)); 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 index 24a258859..81a92798d 100644 --- a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java @@ -248,7 +248,7 @@ public DataTypesTestingPOJO() { groupBitmapUint64 = ClickHouseBitmap.wrap(random.longs(5, Long.MAX_VALUE - 100, Long.MAX_VALUE).toArray()); intervalYear = Period.ofYears(random.nextInt(4000)); - intervalQuarter = Period.ofDays(random.nextInt(1000)); + intervalQuarter = Period.ofMonths(random.nextInt(10) *3); intervalMonth = Period.ofMonths(random.nextInt(12)); intervalWeek = Period.ofWeeks(random.nextInt(52)); intervalDay = Period.ofDays(random.nextInt(30)); @@ -256,7 +256,7 @@ public DataTypesTestingPOJO() { intervalMinute = Duration.ofMinutes(random.nextInt(60)); intervalSecond = Duration.ofSeconds(random.nextInt(60)); intervalMillisecond = Duration.ofMillis(random.nextInt()); - intervalMicrosecond = Duration.ofNanos(random.nextInt()); + intervalMicrosecond = Duration.ofNanos(random.nextInt() * 1000L); intervalNanosecond = Duration.ofNanos((random.nextInt()));