diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index d198e2133..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.6", "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"] @@ -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 @@ -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/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/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java index d723c65d6..c3173d97e 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java @@ -640,8 +640,13 @@ 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(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 @@ private ClickHouseColumn(ClickHouseDataType dataType, String columnName, String 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 49ca5cf7d..885237021 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -1,19 +1,19 @@ package com.clickhouse.data; -import java.lang.reflect.Array; import java.math.BigDecimal; import java.math.BigInteger; import java.net.Inet4Address; import java.net.Inet6Address; +import java.time.Duration; import java.time.LocalDate; import java.time.LocalDateTime; +import java.time.Period; 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; @@ -49,74 +49,75 @@ */ @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, 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"), + 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), - Variant(List.class, true, true, false, 0, 0, 0, 0, 0, true), - + Variant(List.class, true, true, false, 0, 0, 0, 0, 0, true, 0x2A), + Dynamic(Object.class, true, true, false, 0, 0, 0, 0, 0, true, 0x2B), ; public static final List ORDERED_BY_RANGE_INT_TYPES = @@ -212,6 +213,21 @@ 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)); + + 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; } @@ -219,6 +235,66 @@ private static Set> setOf(Class... args) { return Collections.unmodifiableSet(new HashSet<>(Arrays.stream(args).collect(Collectors.toList()))); } + 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 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 IntervalKind { + Nanosecond(IntervalNanosecond, ChronoUnit.NANOS, 0x00), + Microsecond(IntervalMicrosecond, ChronoUnit.MICROS, 0x01), + + Millisecond(IntervalMillisecond, ChronoUnit.MILLIS, 0x02), + + Second(IntervalSecond, ChronoUnit.SECONDS, 0x03), + + Minute(IntervalMinute, ChronoUnit.MINUTES, 0x04), + + Hour(IntervalHour, ChronoUnit.HOURS, 0x05), + + Day(IntervalDay, ChronoUnit.DAYS, 0x06), + + Week(IntervalWeek, ChronoUnit.WEEKS, 0x07), + + Month(IntervalMonth, ChronoUnit.MONTHS, 0x08), + + Quarter(IntervalQuarter, null, 0x09), + + Year(IntervalYear, ChronoUnit.YEARS, 0x1A) // why 1A ? + + ; + + private ClickHouseDataType intervalType; + + private TemporalUnit temporalUnit; + + byte tag; + IntervalKind(ClickHouseDataType clickHouseDataType, TemporalUnit temporalUnit, int tag) { + this.intervalType = clickHouseDataType; + this.tag = (byte) tag; + this.temporalUnit = temporalUnit; + } + + public ClickHouseDataType getIntervalType() { + return intervalType; + } + + public byte getTag() { + return tag; + } + + public TemporalUnit getTemporalUnit() { return temporalUnit; } + } + /** * Immutable set(sorted) for all aliases. @@ -230,6 +306,12 @@ private static Set> setOf(Class... args) { */ public static final Map name2type; + public static final Map binTag2Type; + + public static final Map intervalKind2Type; + + public static final Map intervalType2Kind; + static { Set set = new TreeSet<>(); Map map = new HashMap<>(); @@ -258,6 +340,21 @@ private static Set> setOf(Class... args) { 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<>(); + Map tmpIntervalType2Kind = new HashMap<>(); + for (IntervalKind kind : IntervalKind.values()) { + tmpIntervalKind2Type.put(kind.getTag(), kind.getIntervalType()); + tmpIntervalType2Kind.put(kind.getIntervalType(), kind); + } + intervalKind2Type = Collections.unmodifiableMap(tmpIntervalKind2Type); + intervalType2Kind = Collections.unmodifiableMap(tmpIntervalType2Kind); } /** @@ -490,6 +587,8 @@ public static Class toWiderPrimitiveType(Class javaClass) { private final int maxScale; private final boolean nestedType; + private final byte binTag; + /** * Default constructor. * @@ -521,6 +620,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 { @@ -658,4 +781,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/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseEnum.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseEnum.java index 815676405..877e1d203 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) { @@ -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)) { @@ -135,4 +145,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/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/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/AbstractBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java index e3cb26dfe..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; @@ -247,6 +248,7 @@ protected void setSchema(TableSchema schema) { case Enum8: case Enum16: case Variant: + case Dynamic: this.convertions[i] = NumberConverter.NUMBER_CONVERTERS; break; default: @@ -285,9 +287,11 @@ public static String readAsString(Object value, ClickHouseColumn column) { return zdt.format(com.clickhouse.client.api.DataTypeUtils.DATE_FORMATTER); } 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 @@ -411,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 @@ -603,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 @@ -688,22 +672,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/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 5233d6ec1..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 @@ -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; @@ -16,15 +17,23 @@ 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; +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; @@ -91,12 +100,19 @@ public T readValue(ClickHouseColumn column, Class typeHint) throws IOExce } } + 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 (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); @@ -130,41 +146,41 @@ 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(), 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: return (T) Double.valueOf(readDoubleLE()); case Bool: return (T) Boolean.valueOf(readByteOrEOF(input) == 1); - case Enum8: - return (T) Byte.valueOf((byte) readUnsignedByte()); - case Enum16: - return (T) Short.valueOf((short) readUnsignedShortLE()); + case Enum8: { + byte enum8Val = (byte) readUnsignedByte(); + String name = actualColumn.getEnumConstants().nameNullable(enum8Val); + return (T) new EnumValue(name == null ? "" : name, enum8Val); + } + case Enum16: { + short enum16Val = (short) readUnsignedShortLE(); + String name = actualColumn.getEnumConstants().nameNullable(enum16Val); + return (T) new EnumValue(name == null ? "" : name, enum16Val); + } 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: @@ -176,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)); @@ -198,26 +214,27 @@ 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: - return convertArray(readArray(column), typeHint); + return convertArray(readArray(actualColumn), typeHint); case Map: - return (T) readMap(column); -// case Nested: + return (T) readMap(actualColumn); 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(actualColumn, typeHint); default: - throw new IllegalArgumentException("Unsupported data type: " + column.getDataType()); + throw new IllegalArgumentException("Unsupported data type: " + actualColumn.getDataType()); } } catch (EOFException e) { throw e; @@ -227,13 +244,44 @@ 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(3 * 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(v.longValue() * 1000); + 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; } - 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(); } @@ -244,7 +292,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()) { @@ -458,7 +509,6 @@ public BigDecimal readDecimal(int precision, int scale) throws IOException { } else { v = new BigDecimal(readBigIntegerLE(INT256_SIZE, false), scale); } - return v; } @@ -517,7 +567,7 @@ public static byte[] readNBytesLE(InputStream input, byte[] buffer, int offset, return bytes; } - + /** * Reads a array into an ArrayValue object. * @param column - column information @@ -590,6 +640,8 @@ public static class ArrayValue { final Object array; + int nextPos = 0; + ArrayValue(Class itemType, int length) { this.itemType = itemType; this.length = length; @@ -622,6 +674,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() { @@ -641,6 +698,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 @@ -869,8 +958,7 @@ public static ZonedDateTime readDateTime64(InputStream input, byte[] buff, int s } } - return LocalDateTime.ofInstant(Instant.ofEpochSecond(value, nanoSeconds), tz.toZoneId()) - .atZone(tz.toZoneId()); + return Instant.ofEpochSecond(value, nanoSeconds).atZone(tz.toZoneId()); } /** @@ -956,4 +1044,91 @@ public byte[] allocate(int size) { return new byte[size]; } } + + private static final Set DECIMAL_TAGS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList( + ClickHouseDataType.Decimal.getBinTag(), + ClickHouseDataType.Decimal32.getBinTag(), + ClickHouseDataType.Decimal64.getBinTag(), + ClickHouseDataType.Decimal128.getBinTag(), + ClickHouseDataType.Decimal256.getBinTag() + ))); + + private ClickHouseColumn readDynamicData() throws IOException { + byte tag = readByte(); + + ClickHouseDataType type; + if (tag == ClickHouseDataType.INTERVAL_BIN_TAG) { + byte intervalKind = readByte(); + type = ClickHouseDataType.intervalKind2Type.get(intervalKind); + if (type == null) { + throw new ClientException("Unsupported interval kind: " + intervalKind); + } + return ClickHouseColumn.of("v", type, false, 0, 0); + } else if (tag == ClickHouseDataType.DateTime32.getBinTag()) { + byte scale = readByte(); + return ClickHouseColumn.of("v", "DateTime32(" + scale + ")"); + } else if (tag == ClickHouseDataType.DateTime64.getBinTag() - 1) { // without timezone + byte scale = readByte(); + return ClickHouseColumn.of("v", "DateTime64(" + scale +")"); + } else if (tag == ClickHouseDataType.DateTime64.getBinTag()) { + byte scale = readByte(); + String timezone = readString(input); + return ClickHouseColumn.of("v", "DateTime64(" + scale + (timezone.isEmpty() ? "" : ", " + timezone) +")"); + } else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) { + String typeName = readString(input); + return ClickHouseColumn.of("v", typeName); + } else if (DECIMAL_TAGS.contains(tag)) { + int precision = readByte(); + int scale = readByte(); + return ClickHouseColumn.of("v", ClickHouseDataType.binTag2Type.get(tag), false, precision, scale); + } else if (tag == ClickHouseDataType.Array.getBinTag()) { + ClickHouseColumn elementColumn = readDynamicData(); + return ClickHouseColumn.of("v", "Array(" + elementColumn.getOriginalTypeName() + ")"); + } else if (tag == ClickHouseDataType.Map.getBinTag()) { + ClickHouseColumn keyInfo = readDynamicData(); + ClickHouseColumn valueInfo = readDynamicData(); + return ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + "," + valueInfo.getOriginalTypeName() + ")"); + } else if (tag == ClickHouseDataType.Enum8.getBinTag() || tag == ClickHouseDataType.Enum16.getBinTag()) { + int constants = readVarInt(input); + int[] values = new int[constants]; + String[] names = new String[constants]; + ClickHouseDataType enumType = constants > 127 ? ClickHouseDataType.Enum16 : ClickHouseDataType.Enum8; + for (int i = 0; i < constants; i++) { + names[i] = readString(input); + if (enumType == ClickHouseDataType.Enum8) { + values[i] = readUnsignedByte(); + } else { + values[i] = readUnsignedShortLE(); + } + } + return new ClickHouseColumn(enumType, "v", enumType.name(), false, false, Collections.emptyList(), Collections.emptyList(), + new ClickHouseEnum(names, values)); + } else if (tag == ClickHouseDataType.NULLABLE_BIN_TAG) { + ClickHouseColumn column = readDynamicData(); + return ClickHouseColumn.of("v", "Nullable(" + column.getOriginalTypeName() + ")"); + } else { + type = ClickHouseDataType.binTag2Type.get(tag); + if (type == null) { + throw new ClientException("Unsupported data type with tag " + tag); + } + return ClickHouseColumn.of("v", type, false, 0, 0); + } + } + + 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/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 0423ca739..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 @@ -2,12 +2,11 @@ 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; 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; @@ -31,13 +30,18 @@ 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; import java.util.HashSet; import java.util.List; 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; import java.util.stream.Collectors; @@ -87,6 +91,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); + serializeData(stream, value, typeColumn); + break; default: serializePrimitiveData(stream, value, column); break; @@ -94,28 +103,320 @@ 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(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(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")); + 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)))")); + + 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); + } - 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; + public static ClickHouseColumn valueToColumnForDynamicType(Object value) { + ClickHouseColumn column; + if (value instanceof ZonedDateTime) { + ZonedDateTime dt = (ZonedDateTime) value; + 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; + int scale; + if (d.precision() > ClickHouseDataType.Decimal128.getMaxScale()) { + decType = "Decimal256"; + 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.Decimal64.getMaxScale(); + } else { + decType = "Decimal32"; + scale = ClickHouseDataType.Decimal32.getMaxScale(); + } + + column = ClickHouseColumn.of("v", decType + "(" + 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 Enum) { + column = enumValue2Column((Enum)value); + } else if (value instanceof List || (value !=null && value.getClass().isArray())) { + column = listValue2Column(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 listValue2Column(Object value) { + + 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"); + } } - writeNonNull(stream); } - serializeData(stream, val, column.getNestedColumns().get(0)); + + if (arrayBaseColumn != null) { + if (hasNulls) { + typeStr.insert(insertPos, "Nullable()"); + insertPos += 9; + } + typeStr.insert(insertPos, arrayBaseColumn.getOriginalTypeName()); + break; + } } - } 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()) { + + column = ClickHouseColumn.of("v", typeStr.toString()); + } else { + column = null; + } + 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 { + + 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: + case Enum16: + stream.write(binTag); + 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: + case Decimal64: + case Decimal128: + case Decimal256: + stream.write(binTag); + BinaryStreamUtils.writeUnsignedInt8(stream, dt.getMaxPrecision()); + BinaryStreamUtils.writeUnsignedInt8(stream, dt.getMaxScale()); + break; + case IntervalNanosecond: + case IntervalMillisecond: + case IntervalSecond: + case IntervalMinute: + case IntervalHour: + case IntervalDay: + case IntervalWeek: + case IntervalMonth: + case IntervalQuarter: + case IntervalYear: + stream.write(binTag); + Byte kindTag = ClickHouseDataType.intervalType2Kind.get(dt).getTag(); + if (kindTag == null) { + throw new ClientException("BUG! No Interval Mapping to a kind tag"); + } + stream.write(kindTag); + break; + case DateTime32: + stream.write(binTag); + 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); + ClickHouseColumn arrayElemColumn = typeColumn.getNestedColumns().get(0); + writeDynamicTypeTag(stream, arrayElemColumn); + break; + case Map: + stream.write(binTag); + // 0x27 + writeDynamicTypeTag(stream, typeColumn.getKeyInfo()); + writeDynamicTypeTag(stream, typeColumn.getValueInfo()); + 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); + BinaryStreamUtils.writeString(stream, dt.name()); + 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 == null) { + writeVarInt(stream, 0); + return; + } + + 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; @@ -240,12 +541,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); @@ -259,17 +554,92 @@ private static void serializePrimitiveData(OutputStream stream, Object value, Cl case JSON: serializeJSON(stream, value); break; + case IntervalNanosecond: + case IntervalMicrosecond: + case IntervalMillisecond: + case IntervalSecond: + case IntervalMinute: + case IntervalHour: + case IntervalDay: + case IntervalWeek: + case IntervalMonth: + case IntervalQuarter: + case IntervalYear: + serializeInterval(stream, column, 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()); } } + 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 IntervalWeek: + v = (p.toTotalMonths() * 30 + p.getDays()) / 7; + break; + case IntervalMonth: + v = p.toTotalMonths() + p.getDays() / 30; + break; + case IntervalQuarter: + v = (p.toTotalMonths() + (p.getDays() / 30)) / 3; + break; + case IntervalYear: + v = (p.toTotalMonths() + (p.getDays() / 30)) / 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) { 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()); } @@ -485,7 +855,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 bb4beef64..265e9bd26 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 @@ -447,7 +447,7 @@ public ClassicHttpResponse executeRequest(ClickHouseNode server, Map { 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()); }); } @@ -144,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; } @@ -153,7 +151,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 +158,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; @@ -196,19 +183,19 @@ 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; 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)); @@ -240,8 +227,27 @@ 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; + } - System.out.println("field: " + strValue + " value " + value); if (value.getClass().isPrimitive()) { Assert.assertEquals(strValue, String.valueOf(value)); } else { @@ -388,8 +394,225 @@ public void testVariantWithTuple() throws Exception { }); } + @Test(groups = {"integration"}) + public void testDynamicWithPrimitives() throws Exception { + if (isVersionMatch("(,24.8]") || isCloud()) { + 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("allow_experimental_dynamic_type", "1")); + client.register(DTOForDynamicPrimitivesTests.class, client.getTableSchema(table)); + + int rowId = 0; + for (ClickHouseDataType dataType : ClickHouseDataType.values()) { + switch (dataType) { + case Array: + case Map: + case AggregateFunction: + case SimpleAggregateFunction: + // tested separately + continue; + case Dynamic: + 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; + 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)); + 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"); + 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; + case Decimal32: + case Decimal64: + case Decimal128: + case Decimal256: + 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)); + } else { + Assert.assertEquals(strValue, String.valueOf(value)); + } + } + } + + @Test(groups = {"integration"}) + public void testDynamicWithArrays() throws Exception { + testDynamicWith("arrays", + new Object[]{ + "a,b", + new String[]{"a", null, "b"}, + Arrays.asList("c", "d"), + new Integer[]{1, null, 2, null, 3} + + }, + new String[]{ + "a,b", + "[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(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]", + "[[1, 2], [3, 4]]", + "[[1, 2], []]", + "[[1, 2], [], [3, 4]]" + }); + } + + @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 { + private int rowId; + private Object field; + } + + private void testDynamicWith(String withWhat, Object[] values, String[] expectedStrValues) throws Exception { + if (isVersionMatch("(,24.8]") || isCloud()) { + 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) { + 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]")) { + if (isVersionMatch("(,24.8]") || isCloud()) { return; } @@ -410,7 +633,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")]); } } @@ -426,7 +648,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/datatypes/DataTypesTestingPOJO.java b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java index 90b6594ce..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 @@ -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,6 +96,32 @@ public class DataTypesTestingPOJO { private ClickHouseBitmap groupBitmapUint32; private ClickHouseBitmap groupBitmapUint64; + private Period intervalYear; + + private Period intervalQuarter; + + private Period intervalMonth; + + private Period intervalWeek; + + private Period intervalDay; + + private Duration intervalHour; + + private Duration intervalMinute; + + private Duration intervalSecond; + + private Duration intervalMillisecond; + + private Duration intervalMicrosecond; + + private Duration intervalNanosecond; + + private SmallEnum smallEnum; + + private LargeEnum largeEnum; + public DataTypesTestingPOJO() { final Random random = new Random(); byteValue = (byte) random.nextInt(); @@ -219,6 +247,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 = Period.ofYears(random.nextInt(4000)); + 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)); + 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() * 1000L); + + 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))); } public boolean getBool() { @@ -277,7 +320,51 @@ 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 IntervalDay, " + + "intervalQuarter IntervalDay, " + + "intervalMonth IntervalDay, " + + "intervalWeek IntervalDay, " + + "intervalDay IntervalDay, " + + "intervalHour IntervalHour, " + + "intervalMinute IntervalNanosecond, " + + "intervalSecond IntervalNanosecond, " + + "intervalMillisecond IntervalNanosecond, " + + "intervalMicrosecond IntervalNanosecond, " + + "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 + } } 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 d323f2e8a..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 @@ -32,6 +32,7 @@ import org.apache.commons.compress.compressors.snappy.SnappyCompressorOutputStream; import org.apache.commons.lang3.StringEscapeUtils; import org.testcontainers.shaded.org.apache.commons.lang3.RandomStringUtils; +import org.testcontainers.shaded.org.checkerframework.checker.units.qual.A; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -46,6 +47,8 @@ import java.io.PrintWriter; import java.nio.charset.StandardCharsets; import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; import java.time.ZoneId; import java.time.ZonedDateTime; import java.time.temporal.ChronoField; @@ -53,6 +56,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; @@ -423,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 + "\" " + @@ -606,7 +610,6 @@ public void testAppCompression(String algo) throws Exception { snappy.write(bytes); snappy.finish(); } - System.out.println("Compressed size " + baos.size() + ", uncompressed size: " + data[i].length()); compressedData[i] = baos.toByteArray(); } @@ -614,7 +617,6 @@ public void testAppCompression(String algo) throws Exception { .appCompressedData(true, algo); try (InsertResponse response = client.insert(tableName, out -> { for (byte[] row : compressedData) { -// if (algo.) BinaryStreamUtils.writeVarInt(out, row.length); out.write(row); } @@ -638,8 +640,57 @@ 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); + + 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); + } + + 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/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 4d605068f..59a457c90 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,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; @@ -1418,6 +1417,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,)") && !isCloud()) { + settings.serverSetting("allow_experimental_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,7 +1429,7 @@ 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(), settings).get(10, TimeUnit.SECONDS); // Insert data StringBuilder insertStmtBuilder = new StringBuilder(); @@ -1784,9 +1788,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; } @@ -1822,9 +1824,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; } @@ -1906,9 +1906,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(); @@ -2047,8 +2045,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; } @@ -2059,7 +2056,78 @@ public void testGettingRowsBeforeLimit() throws Exception { Assert.assertEquals(response.getTotalRowsToRead(), expectedTotalRowsToRead); } } - + + @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); + 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); + 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 testBinaryEncodedJSON() throws Exception { + if (isVersionMatch("(,24.8]")) { + return; + } + + 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); + + 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()); + } + } + } + } + + public boolean isVersionMatch(String versionExpression) { + List serverVersion = client.queryAll("SELECT version()"); + return ClickHouseVersion.of(serverVersion.get(0).getString(1)).check(versionExpression); + } + @Test(groups = {"integration"}) public void testEmptyResponse() throws Exception { try (QueryResponse response = client.query("SELECT number FROM system.numbers LIMIT 0", new QuerySettings().setFormat(ClickHouseFormat.RowBinary)).get()) { 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..b3a9f5c5b 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,20 @@ public void testGeometricTypesSimpleStatement() throws SQLException { } - @Test (enabled = false)//TODO: This type is experimental right now + @Test(groups = { "integration" }) public void testDynamicTypesSimpleStatement() throws SQLException { + if (earlierThan(24, 8) || isCloud()) { + return; + } + + Properties properties = new Properties(); + if (!isCloud()) { + 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 +938,52 @@ public void testTypeConversions() throws Exception { } } } + + @Test(groups = { "integration" }) + public void testVariantTypesSimpleStatement() throws SQLException { + if (earlierThan(24, 8) || isCloud()) { + return; + } + + Properties properties = new Properties(); + 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 ()", + 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); }