From e0590c3fc6985e8a9fcefd90f5012c2478a6f745 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 10 Jun 2026 10:49:43 -0700 Subject: [PATCH 1/3] Added reader for JSONEachRow format. Updated documentation and examples --- .../clickhouse/data/ClickHouseDataType.java | 2 +- client-v2/pom.xml | 20 +- .../com/clickhouse/client/api/Client.java | 25 + .../client/api/ClientConfigProperties.java | 5 + .../ClickHouseBinaryFormatReader.java | 633 +-------- .../data_formats/ClickHouseFormatReader.java | 629 ++++++++ .../ClickHouseTextFormatReader.java | 46 + .../data_formats/GsonJsonParserFactory.java | 68 + .../data_formats/JSONEachRowFormatReader.java | 645 +++++++++ .../JacksonJsonParserFactory.java | 51 + .../client/api/data_formats/JsonParser.java | 41 + .../api/data_formats/JsonParserFactory.java | 17 + .../client/api/internal/SchemaUtils.java | 87 ++ .../com/clickhouse/client/ClientTests.java | 6 +- .../AbstractJSONEachRowFormatReaderTests.java | 1261 +++++++++++++++++ .../GsonJSONEachRowFormatReaderTests.java | 74 + .../JSONEachRowFormatReaderTest.java | 795 +++++++++++ .../JacksonJSONEachRowFormatReaderTests.java | 23 + .../internal/JacksonJsonParserTest.java | 102 ++ .../clickhouse/client/query/QueryTests.java | 39 + docs/client-v2-json-support.md | 644 +++++++++ docs/features.md | 6 +- examples/client-v2-json-processors/.gitignore | 2 + examples/client-v2-json-processors/README.md | 138 ++ .../build.gradle.kts | 37 + .../gradle.properties | 1 + .../gradle/libs.versions.toml | 12 + .../gradle/wrapper/gradle-wrapper.properties | 7 + examples/client-v2-json-processors/gradlew | 248 ++++ .../client-v2-json-processors/gradlew.bat | 93 ++ .../settings.gradle.kts | 5 + .../ClientV2JsonProcessorsExample.java | 283 ++++ examples/jdbc-dispatcher-demo/.gitignore | 2 + examples/jdbc-v2-json-processors/.gitignore | 2 + examples/jdbc-v2-json-processors/README.md | 193 +++ .../jdbc-v2-json-processors/build.gradle.kts | 37 + .../jdbc-v2-json-processors/gradle.properties | 1 + .../gradle/libs.versions.toml | 12 + .../gradle/wrapper/gradle-wrapper.properties | 7 + examples/jdbc-v2-json-processors/gradlew | 248 ++++ examples/jdbc-v2-json-processors/gradlew.bat | 93 ++ .../settings.gradle.kts | 5 + .../JdbcV2JsonProcessorsExample.java | 329 +++++ jdbc-v2/pom.xml | 39 +- .../com/clickhouse/jdbc/ConnectionImpl.java | 61 +- .../com/clickhouse/jdbc/DriverProperties.java | 8 + .../com/clickhouse/jdbc/ResultSetImpl.java | 26 +- .../com/clickhouse/jdbc/StatementImpl.java | 21 +- .../clickhouse/jdbc/ResultSetImplTest.java | 46 + .../com/clickhouse/jdbc/StatementTest.java | 41 + .../jdbc/internal/JdbcConfigurationTest.java | 3 +- 51 files changed, 6571 insertions(+), 648 deletions(-) create mode 100644 client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseFormatReader.java create mode 100644 client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseTextFormatReader.java create mode 100644 client-v2/src/main/java/com/clickhouse/client/api/data_formats/GsonJsonParserFactory.java create mode 100644 client-v2/src/main/java/com/clickhouse/client/api/data_formats/JSONEachRowFormatReader.java create mode 100644 client-v2/src/main/java/com/clickhouse/client/api/data_formats/JacksonJsonParserFactory.java create mode 100644 client-v2/src/main/java/com/clickhouse/client/api/data_formats/JsonParser.java create mode 100644 client-v2/src/main/java/com/clickhouse/client/api/data_formats/JsonParserFactory.java create mode 100644 client-v2/src/main/java/com/clickhouse/client/api/internal/SchemaUtils.java create mode 100644 client-v2/src/test/java/com/clickhouse/client/api/data_formats/AbstractJSONEachRowFormatReaderTests.java create mode 100644 client-v2/src/test/java/com/clickhouse/client/api/data_formats/GsonJSONEachRowFormatReaderTests.java create mode 100644 client-v2/src/test/java/com/clickhouse/client/api/data_formats/JSONEachRowFormatReaderTest.java create mode 100644 client-v2/src/test/java/com/clickhouse/client/api/data_formats/JacksonJSONEachRowFormatReaderTests.java create mode 100644 client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/JacksonJsonParserTest.java create mode 100644 docs/client-v2-json-support.md create mode 100644 examples/client-v2-json-processors/.gitignore create mode 100644 examples/client-v2-json-processors/README.md create mode 100644 examples/client-v2-json-processors/build.gradle.kts create mode 100644 examples/client-v2-json-processors/gradle.properties create mode 100644 examples/client-v2-json-processors/gradle/libs.versions.toml create mode 100644 examples/client-v2-json-processors/gradle/wrapper/gradle-wrapper.properties create mode 100755 examples/client-v2-json-processors/gradlew create mode 100644 examples/client-v2-json-processors/gradlew.bat create mode 100644 examples/client-v2-json-processors/settings.gradle.kts create mode 100644 examples/client-v2-json-processors/src/main/java/com/clickhouse/examples/client_v2/json_processors/ClientV2JsonProcessorsExample.java create mode 100644 examples/jdbc-dispatcher-demo/.gitignore create mode 100644 examples/jdbc-v2-json-processors/.gitignore create mode 100644 examples/jdbc-v2-json-processors/README.md create mode 100644 examples/jdbc-v2-json-processors/build.gradle.kts create mode 100644 examples/jdbc-v2-json-processors/gradle.properties create mode 100644 examples/jdbc-v2-json-processors/gradle/libs.versions.toml create mode 100644 examples/jdbc-v2-json-processors/gradle/wrapper/gradle-wrapper.properties create mode 100755 examples/jdbc-v2-json-processors/gradlew create mode 100644 examples/jdbc-v2-json-processors/gradlew.bat create mode 100644 examples/jdbc-v2-json-processors/settings.gradle.kts create mode 100644 examples/jdbc-v2-json-processors/src/main/java/com/clickhouse/examples/jdbc_v2/json_processors/JdbcV2JsonProcessorsExample.java diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java index c93963fa1..9199e64b5 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseDataType.java @@ -177,7 +177,7 @@ public static Map, Integer> buildVariantMapping(List>> DATA_TYPE_TO_CLASS = dataTypeClassMap(); + public static final Map>> DATA_TYPE_TO_CLASS = Collections.unmodifiableMap(dataTypeClassMap()); static Map>> dataTypeClassMap() { Map>> map = new HashMap<>(); diff --git a/client-v2/pom.xml b/client-v2/pom.xml index 0c6409cdf..57ff943a8 100644 --- a/client-v2/pom.xml +++ b/client-v2/pom.xml @@ -88,8 +88,26 @@ com.fasterxml.jackson.core jackson-databind - test ${jackson.version} + provided + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + provided + + + com.fasterxml.jackson.core + jackson-annotations + ${jackson.version} + provided + + + com.google.code.gson + gson + ${gson.version} + provided ${project.parent.groupId} diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java index 58d94da9b..211288a30 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java @@ -1651,6 +1651,7 @@ public CompletableFuture query(String sqlQuery, Map buildRequestSettings(Map opSettings) return requestSettings; } + /** + * Applies format-specific server-side settings to the already merged request settings. + * Must be called after {@link #buildRequestSettings(Map)} and after the request format has been resolved + * (either provided by the caller or defaulted), so that the inspected format reflects the final value. + * + *

For {@link ClickHouseFormat#JSONEachRow}, callers may opt in to plain JSON numbers by setting + * {@link ClientConfigProperties#JSON_DISABLE_NUMBER_QUOTING}. Explicit server settings are otherwise + * left untouched.

+ *
    + *
  • {@code output_format_json_quote_64bit_integers}
  • + *
  • {@code output_format_json_quote_64bit_floats}
  • + *
  • {@code output_format_json_quote_decimals}
  • + *
+ */ + private static void applyFormatSpecificSettings(QuerySettings requestSettings) { + boolean disableNumberQuoting = ClientConfigProperties.JSON_DISABLE_NUMBER_QUOTING + .getOrDefault(requestSettings.getAllSettings()); + if (requestSettings.getFormat() == ClickHouseFormat.JSONEachRow && disableNumberQuoting) { + requestSettings.serverSetting("output_format_json_quote_64bit_integers", "0"); + requestSettings.serverSetting("output_format_json_quote_64bit_floats", "0"); + requestSettings.serverSetting("output_format_json_quote_decimals", "0"); + } + } + private Duration durationSince(long sinceNanos) { return Duration.ofNanos(System.nanoTime() - sinceNanos); } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java b/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java index e548a90f9..5a61fcebf 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java @@ -190,6 +190,11 @@ public Object parseValue(String value) { */ HTTP_SEND_PARAMS_IN_BODY("client.http.use_form_request_for_query", Boolean.class, "false"), + /** + * When enabled for JSONEachRow queries, asks ClickHouse to emit large integer, + * floating-point, and decimal values as JSON numbers instead of quoted strings. + */ + JSON_DISABLE_NUMBER_QUOTING("json_disable_number_quoting", Boolean.class, "false"), /** * Prefix for custom settings. Should be aligned with server configuration. 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 51e1b1df0..c95bde792 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 @@ -1,618 +1,21 @@ package com.clickhouse.client.api.data_formats; -import com.clickhouse.client.api.metadata.TableSchema; -import com.clickhouse.data.value.ClickHouseBitmap; -import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; -import com.clickhouse.data.value.ClickHouseGeoPointValue; -import com.clickhouse.data.value.ClickHouseGeoPolygonValue; -import com.clickhouse.data.value.ClickHouseGeoRingValue; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.net.Inet4Address; -import java.net.Inet6Address; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZonedDateTime; -import java.time.temporal.TemporalAmount; -import java.util.List; -import java.util.Map; -import java.util.UUID; - -public interface ClickHouseBinaryFormatReader extends AutoCloseable { - - /** - * Reads a single value from the stream. - * - * @param - * @return - */ - T readValue(int colIndex); - - /** - * Reads a row to an array of objects. - * - * @param colName - * @param - * @return - */ - T readValue(String colName); - - boolean hasValue(String colName); - - boolean hasValue(int colIndex); - - /** - * Checks if there are more rows to read. - * - * @return - */ - boolean hasNext(); - - /** - * Moves cursor to the next row. Must be called before reading the first row. Returns reference to - * an internal record representation. It means that next call to the method will affect value in returned Map. - * This is done for memory usage optimization. - * Method is intended to be used only by the client not an application. - * - * @return reference to a map filled with column values or null if no more records are available - */ - Map next(); - - /** - * Reads column with name `colName` as a string. - * - * @param colName - column name - * @return - */ - String getString(String colName); - - /** - * Reads column with name `colName` as a byte. - * - * @param colName - column name - * @return - */ - byte getByte(String colName); - - /** - * Reads column with name `colName` as a short. - * - * @param colName - column name - * @return - */ - short getShort(String colName); - - /** - * Reads column with name `colName` as an integer. - * - * @param colName - column name - * @return - */ - int getInteger(String colName); - - /** - * Reads column with name `colName` as a long. - * - * @param colName - column name - * @return - */ - long getLong(String colName); - - /** - * Reads column with name `colName` as a float. - * Warning: this method may lose precision for float values. - * - * @param colName - * @return - */ - float getFloat(String colName); - - /** - * Reads column with name `colName` as a double. - * Warning: this method may lose precision for double values. - * - * @param colName - * @return - */ - double getDouble(String colName); - - /** - * Reads column with name `colName` as a boolean. - * - * @param colName - * @return - */ - boolean getBoolean(String colName); - - /** - * Reads column with name `colName` as a BigInteger. - * - * @param colName - * @return - */ - BigInteger getBigInteger(String colName); - - /** - * Reads column with name `colName` as a BigDecimal. - * - * @param colName - * @return - */ - BigDecimal getBigDecimal(String colName); - - /** - * Returns the value of the specified column as an Instant. Timezone is derived from the column definition. - * If no timezone is specified in the column definition then UTC will be used. - * - * If column value is Date or Date32 it will return an Instant with time set to 00:00:00. - * If column value is DateTime or DateTime32 it will return an Instant with the time part. - * - * @param colName - * @return - */ - Instant getInstant(String colName); - - /** - * Returns the value of the specified column as a ZonedDateTime. Timezone is derived from the column definition. - * If no timezone is specified in the column definition then UTC will be used. - * - * If column value is Date or Date32 it will return a ZonedDateTime with time set to 00:00:00. - * If column value is DateTime or DateTime32 it will return a ZonedDateTime with the time part. - * - * @param colName - * @return - */ - ZonedDateTime getZonedDateTime(String colName); - - /** - * Returns the value of the specified column as a Duration. - * - * If a stored value is bigger than Long.MAX_VALUE then exception will be thrown. In such case - * use asBigInteger() method. - * - * If value of IntervalQuarter then Duration will be in the unit of Months. - * - * @param colName - * @return Duration in the unit of column type. - */ - Duration getDuration(String colName); - - - /** - * Returns the value of the specified column as an Inet4Address. - * - * @param colName - * @return - */ - Inet4Address getInet4Address(String colName); - - /** - * Returns the value of the specified column as an Inet6Address. - * - * @param colName - * @return - */ - Inet6Address getInet6Address(String colName); - - /** - * Returns the value of the specified column as a UUID. - * - * @param colName - * @return - */ - UUID getUUID(String colName); - - /** - * Returns the value of the specified column as a ClickHouseGeoPointValue. - * - * @param colName - * @return - */ - ClickHouseGeoPointValue getGeoPoint(String colName); - - /** - * Returns the value of the specified column as a ClickHouseGeoRingValue. - * - * @param colName - * @return - */ - ClickHouseGeoRingValue getGeoRing(String colName); - - /** - * Returns the value of the specified column as a ClickHouseGeoPolygonValue. - * - * @param colName - * @return - */ - ClickHouseGeoPolygonValue getGeoPolygon(String colName); - - /** - * Returns the value of the specified column as a ClickHouseGeoMultiPolygonValue. - * - * @param colName - * @return - */ - ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName); - - /** - * @see #getList(int) - * @param colName - column name - * @return list of values, or {@code null} if the value is null - */ - List getList(String colName); - - /** - * @see #getByteArray(int) - * @param colName - column name - * @return array of bytes, or {@code null} if the value is null - */ - byte[] getByteArray(String colName); - - /** - * @see #getIntArray(int) - * @param colName - column name - * @return array of int values, or {@code null} if the value is null - */ - int[] getIntArray(String colName); - - /** - * @see #getLongArray(int) - * @param colName - column name - * @return array of long values, or {@code null} if the value is null - */ - long[] getLongArray(String colName); - - /** - * @see #getFloatArray(int) - * @param colName - column name - * @return array of float values, or {@code null} if the value is null - */ - float[] getFloatArray(String colName); - - /** - * @see #getDoubleArray(int) - * @param colName - column name - * @return array of double values, or {@code null} if the value is null - */ - double[] getDoubleArray(String colName); - - /** - * @see #getBooleanArray(int) - * @param colName - column name - * @return array of boolean values, or {@code null} if the value is null - */ - boolean[] getBooleanArray(String colName); - - /** - * @see #getShortArray(int) - * @param colName - column name - * @return array of short values, or {@code null} if the value is null - */ - short[] getShortArray(String colName); - - /** - * @see #getStringArray(int) - * @param colName - column name - * @return array of string values, or {@code null} if the value is null - */ - String[] getStringArray(String colName); - - /** - * @see #getObjectArray(int) - * @param colName - column name - * @return array of objects, or {@code null} if the value is null - */ - Object[] getObjectArray(String colName); - - /** - * Reads column with name `colName` as a string. - * - * @param index - * @return - */ - String getString(int index); - - /** - * Reads column with name `colName` as a byte. - * - * @param index - * @return - */ - byte getByte(int index); - - /** - * Reads column with name `colName` as a short. - * - * @param index - * @return - */ - short getShort(int index); - - /** - * Reads column with name `colName` as an integer. - * - * @param index - * @return - */ - int getInteger(int index); - - /** - * Reads column with name `colName` as a long. - * - * @param index - * @return - */ - long getLong(int index); - - /** - * Reads column with name `colName` as a float. - * Warning: this method may lose precision for float values. - * - * @param index - * @return - */ - float getFloat(int index); - - /** - * Reads column with name `colName` as a double. - * Warning: this method may lose precision for double values. - * - * @param index - * @return - */ - double getDouble(int index); - - /** - * Reads column with name `colName` as a boolean. - * - * @param index - * @return - */ - boolean getBoolean(int index); - - /** - * Reads column with name `colName` as a BigInteger. - * - * @param index - * @return - */ - BigInteger getBigInteger(int index); - - /** - * Reads column with name `colName` as a BigDecimal. - * - * @param index - * @return - */ - BigDecimal getBigDecimal(int index); - - /** - * Returns the value of the specified column as an Instant. Timezone is derived from the column definition. - * If no timezone is specified in the column definition then UTC will be used. - * - * If column value is Date or Date32 it will return an Instant with time set to 00:00:00. - * If column value is DateTime or DateTime32 it will return an Instant with the time part. - * - * @param index - * @return - */ - Instant getInstant(int index); - - /** - * Returns the value of the specified column as a ZonedDateTime. Timezone is derived from the column definition. - * If no timezone is specified in the column definition then UTC will be used. - * - * If column value is Date or Date32 it will return a ZonedDateTime with time set to 00:00:00. - * If column value is DateTime or DateTime32 it will return a ZonedDateTime with the time part. - * - * @param index - * @return - */ - ZonedDateTime getZonedDateTime(int index); - - /** - * Returns the value of the specified column as a Duration. - * If a stored value is bigger than Long.MAX_VALUE then exception will be thrown. In such case - * use asBigInteger() method. - * If value of IntervalQuarter then Duration will be in the unit of Months. - * - * @param index - * @return Duration in the unit of column type. - */ - Duration getDuration(int index); - - - /** - * Returns the value of the specified column as an Inet4Address. - * - * @param index - * @return - */ - Inet4Address getInet4Address(int index); - - /** - * Returns the value of the specified column as an Inet6Address. - * - * @param index - * @return - */ - Inet6Address getInet6Address(int index); - - /** - * Returns the value of the specified column as a UUID. - * - * @param index - * @return - */ - UUID getUUID(int index); - - /** - * Returns the value of the specified column as a ClickHouseGeoPointValue. - * - * @param index - * @return - */ - ClickHouseGeoPointValue getGeoPoint(int index); - - /** - * Returns the value of the specified column as a ClickHouseGeoRingValue. - * - * @param index - * @return - */ - ClickHouseGeoRingValue getGeoRing(int index); - - /** - * Returns the value of the specified column as a ClickHouseGeoPolygonValue. - * - * @param index - * @return - */ - ClickHouseGeoPolygonValue getGeoPolygon(int index); - - /** - * Returns the value of the specified column as a ClickHouseGeoMultiPolygonValue. - * - * @param index - * @return - */ - ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index); - - /** - * Returns the value of the specified column as a {@link List}. Suitable for reading Array columns of any type. - *

For nested arrays (e.g. {@code Array(Array(Int64))}), returns a {@code List>}. - * For nullable arrays (e.g. {@code Array(Nullable(Int32))}), list elements may be {@code null}.

- * - * @param index - column index (1-based) - * @return list of values, or {@code null} if the value is null - * @throws com.clickhouse.client.api.ClientException if the column is not an array type - */ - List getList(int index); - - /** - * Returns the value of the specified column as a {@code byte[]}. Suitable for 1D Array columns only. - * - * @param index - column index (1-based) - * @return array of bytes, or {@code null} if the value is null - * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a byte array - */ - byte[] getByteArray(int index); - - /** - * Returns the value of the specified column as an {@code int[]}. Suitable for 1D Array columns only. - * - * @param index - column index (1-based) - * @return array of int values, or {@code null} if the value is null - * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to an int array - */ - int[] getIntArray(int index); - - /** - * Returns the value of the specified column as a {@code long[]}. Suitable for 1D Array columns only. - * - * @param index - column index (1-based) - * @return array of long values, or {@code null} if the value is null - * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a long array - */ - long[] getLongArray(int index); - - /** - * Returns the value of the specified column as a {@code float[]}. Suitable for 1D Array columns only. - * - * @param index - column index (1-based) - * @return array of float values, or {@code null} if the value is null - * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a float array - */ - float[] getFloatArray(int index); - - /** - * Returns the value of the specified column as a {@code double[]}. Suitable for 1D Array columns only. - * - * @param index - column index (1-based) - * @return array of double values, or {@code null} if the value is null - * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a double array - */ - double[] getDoubleArray(int index); - - /** - * Returns the value of the specified column as a {@code boolean[]}. Suitable for 1D Array columns only. - * - * @param index - column index (1-based) - * @return array of boolean values, or {@code null} if the value is null - * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a boolean array - */ - boolean[] getBooleanArray(int index); - - /** - * Returns the value of the specified column as a {@code short[]}. Suitable for 1D Array columns only. - * - * @param index - column index (1-based) - * @return array of short values, or {@code null} if the value is null - * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a short array - */ - short[] getShortArray(int index); - - /** - * Returns the value of the specified column as a {@code String[]}. Suitable for 1D Array columns only. - * Cannot be used for none string element types. - * - * @param index - column index (1-based) - * @return array of string values, or {@code null} if the value is null - * @throws com.clickhouse.client.api.ClientException if the column is not an array type - */ - String[] getStringArray(int index); - - /** - * Returns the value of the specified column as an {@code Object[]}. Suitable for multidimensional Array columns. - * Nested arrays are recursively converted to {@code Object[]}. - * Note: result is not cached so avoid repetitive calls on same column. - * - * @param index - column index (1-based) - * @return array of objects, or {@code null} if the value is null - * @throws com.clickhouse.client.api.ClientException if the column is not an array type - */ - Object[] getObjectArray(int index); - - Object[] getTuple(int index); - - Object[] getTuple(String colName); - - byte getEnum8(String colName); - - byte getEnum8(int index); - - short getEnum16(String colName); - - short getEnum16(int index); - - LocalDate getLocalDate(String colName); - - LocalDate getLocalDate(int index); - - LocalTime getLocalTime(String colName); - - LocalTime getLocalTime(int index); - - LocalDateTime getLocalDateTime(String colName); - - LocalDateTime getLocalDateTime(int index); - - OffsetDateTime getOffsetDateTime(String colName); - - OffsetDateTime getOffsetDateTime(int index); - - TableSchema getSchema(); - - ClickHouseBitmap getClickHouseBitmap(String colName); - - ClickHouseBitmap getClickHouseBitmap(int index); - - TemporalAmount getTemporalAmount(int index); - - TemporalAmount getTemporalAmount(String colName); +/** + * Reader for ClickHouse binary output formats (such as {@code Native}, + * {@code RowBinary}, {@code RowBinaryWithNames}, and + * {@code RowBinaryWithNamesAndTypes}). + * + *

Row navigation, schema access, and typed accessors are inherited from + * {@link ClickHouseFormatReader}; this interface specializes the contract for + * binary-encoded result streams and is the type returned by the binary + * factory methods on {@link com.clickhouse.client.api.Client}. Readers for + * text-oriented output formats (for example {@code JSONEachRow}) implement + * {@link ClickHouseTextFormatReader} instead.

+ * + *

Instances are produced by + * {@link com.clickhouse.client.api.Client#newBinaryFormatReader(com.clickhouse.client.api.query.QueryResponse)} + * and + * {@link com.clickhouse.client.api.Client#newBinaryFormatReader(com.clickhouse.client.api.query.QueryResponse, com.clickhouse.client.api.metadata.TableSchema)}.

+ */ +public interface ClickHouseBinaryFormatReader extends ClickHouseFormatReader { } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseFormatReader.java new file mode 100644 index 000000000..2464798fe --- /dev/null +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseFormatReader.java @@ -0,0 +1,629 @@ +package com.clickhouse.client.api.data_formats; + +import com.clickhouse.client.api.metadata.TableSchema; +import com.clickhouse.data.value.ClickHouseBitmap; +import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoPointValue; +import com.clickhouse.data.value.ClickHouseGeoPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoRingValue; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.Inet4Address; +import java.net.Inet6Address; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZonedDateTime; +import java.time.temporal.TemporalAmount; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +/** + * Common contract for row-by-row format readers regardless of the underlying + * wire encoding. Implementations are produced for binary formats (see + * {@link ClickHouseBinaryFormatReader}) and text formats (see + * {@link ClickHouseTextFormatReader}). + * + *

The methods declared here describe row navigation, schema access, and + * typed accessors by column name and 1-based column index. Concrete readers + * may not support every accessor for every format; unsupported accessors are + * expected to throw {@link UnsupportedOperationException}.

+ */ +public interface ClickHouseFormatReader extends AutoCloseable { + + /** + * Reads a single value from the stream. + * + * @param + * @return + */ + T readValue(int colIndex); + + /** + * Reads a row to an array of objects. + * + * @param colName + * @param + * @return + */ + T readValue(String colName); + + boolean hasValue(String colName); + + boolean hasValue(int colIndex); + + /** + * Checks if there are more rows to read. + * + * @return + */ + boolean hasNext(); + + /** + * Moves cursor to the next row. Must be called before reading the first row. Returns reference to + * an internal record representation. It means that next call to the method will affect value in returned Map. + * This is done for memory usage optimization. + * Method is intended to be used only by the client not an application. + * + * @return reference to a map filled with column values or null if no more records are available + */ + Map next(); + + /** + * Reads column with name `colName` as a string. + * + * @param colName - column name + * @return + */ + String getString(String colName); + + /** + * Reads column with name `colName` as a byte. + * + * @param colName - column name + * @return + */ + byte getByte(String colName); + + /** + * Reads column with name `colName` as a short. + * + * @param colName - column name + * @return + */ + short getShort(String colName); + + /** + * Reads column with name `colName` as an integer. + * + * @param colName - column name + * @return + */ + int getInteger(String colName); + + /** + * Reads column with name `colName` as a long. + * + * @param colName - column name + * @return + */ + long getLong(String colName); + + /** + * Reads column with name `colName` as a float. + * Warning: this method may lose precision for float values. + * + * @param colName + * @return + */ + float getFloat(String colName); + + /** + * Reads column with name `colName` as a double. + * Warning: this method may lose precision for double values. + * + * @param colName + * @return + */ + double getDouble(String colName); + + /** + * Reads column with name `colName` as a boolean. + * + * @param colName + * @return + */ + boolean getBoolean(String colName); + + /** + * Reads column with name `colName` as a BigInteger. + * + * @param colName + * @return + */ + BigInteger getBigInteger(String colName); + + /** + * Reads column with name `colName` as a BigDecimal. + * + * @param colName + * @return + */ + BigDecimal getBigDecimal(String colName); + + /** + * Returns the value of the specified column as an Instant. Timezone is derived from the column definition. + * If no timezone is specified in the column definition then UTC will be used. + * + * If column value is Date or Date32 it will return an Instant with time set to 00:00:00. + * If column value is DateTime or DateTime32 it will return an Instant with the time part. + * + * @param colName + * @return + */ + Instant getInstant(String colName); + + /** + * Returns the value of the specified column as a ZonedDateTime. Timezone is derived from the column definition. + * If no timezone is specified in the column definition then UTC will be used. + * + * If column value is Date or Date32 it will return a ZonedDateTime with time set to 00:00:00. + * If column value is DateTime or DateTime32 it will return a ZonedDateTime with the time part. + * + * @param colName + * @return + */ + ZonedDateTime getZonedDateTime(String colName); + + /** + * Returns the value of the specified column as a Duration. + * + * If a stored value is bigger than Long.MAX_VALUE then exception will be thrown. In such case + * use asBigInteger() method. + * + * If value of IntervalQuarter then Duration will be in the unit of Months. + * + * @param colName + * @return Duration in the unit of column type. + */ + Duration getDuration(String colName); + + + /** + * Returns the value of the specified column as an Inet4Address. + * + * @param colName + * @return + */ + Inet4Address getInet4Address(String colName); + + /** + * Returns the value of the specified column as an Inet6Address. + * + * @param colName + * @return + */ + Inet6Address getInet6Address(String colName); + + /** + * Returns the value of the specified column as a UUID. + * + * @param colName + * @return + */ + UUID getUUID(String colName); + + /** + * Returns the value of the specified column as a ClickHouseGeoPointValue. + * + * @param colName + * @return + */ + ClickHouseGeoPointValue getGeoPoint(String colName); + + /** + * Returns the value of the specified column as a ClickHouseGeoRingValue. + * + * @param colName + * @return + */ + ClickHouseGeoRingValue getGeoRing(String colName); + + /** + * Returns the value of the specified column as a ClickHouseGeoPolygonValue. + * + * @param colName + * @return + */ + ClickHouseGeoPolygonValue getGeoPolygon(String colName); + + /** + * Returns the value of the specified column as a ClickHouseGeoMultiPolygonValue. + * + * @param colName + * @return + */ + ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName); + + /** + * @see #getList(int) + * @param colName - column name + * @return list of values, or {@code null} if the value is null + */ + List getList(String colName); + + /** + * @see #getByteArray(int) + * @param colName - column name + * @return array of bytes, or {@code null} if the value is null + */ + byte[] getByteArray(String colName); + + /** + * @see #getIntArray(int) + * @param colName - column name + * @return array of int values, or {@code null} if the value is null + */ + int[] getIntArray(String colName); + + /** + * @see #getLongArray(int) + * @param colName - column name + * @return array of long values, or {@code null} if the value is null + */ + long[] getLongArray(String colName); + + /** + * @see #getFloatArray(int) + * @param colName - column name + * @return array of float values, or {@code null} if the value is null + */ + float[] getFloatArray(String colName); + + /** + * @see #getDoubleArray(int) + * @param colName - column name + * @return array of double values, or {@code null} if the value is null + */ + double[] getDoubleArray(String colName); + + /** + * @see #getBooleanArray(int) + * @param colName - column name + * @return array of boolean values, or {@code null} if the value is null + */ + boolean[] getBooleanArray(String colName); + + /** + * @see #getShortArray(int) + * @param colName - column name + * @return array of short values, or {@code null} if the value is null + */ + short[] getShortArray(String colName); + + /** + * @see #getStringArray(int) + * @param colName - column name + * @return array of string values, or {@code null} if the value is null + */ + String[] getStringArray(String colName); + + /** + * @see #getObjectArray(int) + * @param colName - column name + * @return array of objects, or {@code null} if the value is null + */ + Object[] getObjectArray(String colName); + + /** + * Reads column with name `colName` as a string. + * + * @param index + * @return + */ + String getString(int index); + + /** + * Reads column with name `colName` as a byte. + * + * @param index + * @return + */ + byte getByte(int index); + + /** + * Reads column with name `colName` as a short. + * + * @param index + * @return + */ + short getShort(int index); + + /** + * Reads column with name `colName` as an integer. + * + * @param index + * @return + */ + int getInteger(int index); + + /** + * Reads column with name `colName` as a long. + * + * @param index + * @return + */ + long getLong(int index); + + /** + * Reads column with name `colName` as a float. + * Warning: this method may lose precision for float values. + * + * @param index + * @return + */ + float getFloat(int index); + + /** + * Reads column with name `colName` as a double. + * Warning: this method may lose precision for double values. + * + * @param index + * @return + */ + double getDouble(int index); + + /** + * Reads column with name `colName` as a boolean. + * + * @param index + * @return + */ + boolean getBoolean(int index); + + /** + * Reads column with name `colName` as a BigInteger. + * + * @param index + * @return + */ + BigInteger getBigInteger(int index); + + /** + * Reads column with name `colName` as a BigDecimal. + * + * @param index + * @return + */ + BigDecimal getBigDecimal(int index); + + /** + * Returns the value of the specified column as an Instant. Timezone is derived from the column definition. + * If no timezone is specified in the column definition then UTC will be used. + * + * If column value is Date or Date32 it will return an Instant with time set to 00:00:00. + * If column value is DateTime or DateTime32 it will return an Instant with the time part. + * + * @param index + * @return + */ + Instant getInstant(int index); + + /** + * Returns the value of the specified column as a ZonedDateTime. Timezone is derived from the column definition. + * If no timezone is specified in the column definition then UTC will be used. + * + * If column value is Date or Date32 it will return a ZonedDateTime with time set to 00:00:00. + * If column value is DateTime or DateTime32 it will return a ZonedDateTime with the time part. + * + * @param index + * @return + */ + ZonedDateTime getZonedDateTime(int index); + + /** + * Returns the value of the specified column as a Duration. + * If a stored value is bigger than Long.MAX_VALUE then exception will be thrown. In such case + * use asBigInteger() method. + * If value of IntervalQuarter then Duration will be in the unit of Months. + * + * @param index + * @return Duration in the unit of column type. + */ + Duration getDuration(int index); + + + /** + * Returns the value of the specified column as an Inet4Address. + * + * @param index + * @return + */ + Inet4Address getInet4Address(int index); + + /** + * Returns the value of the specified column as an Inet6Address. + * + * @param index + * @return + */ + Inet6Address getInet6Address(int index); + + /** + * Returns the value of the specified column as a UUID. + * + * @param index + * @return + */ + UUID getUUID(int index); + + /** + * Returns the value of the specified column as a ClickHouseGeoPointValue. + * + * @param index + * @return + */ + ClickHouseGeoPointValue getGeoPoint(int index); + + /** + * Returns the value of the specified column as a ClickHouseGeoRingValue. + * + * @param index + * @return + */ + ClickHouseGeoRingValue getGeoRing(int index); + + /** + * Returns the value of the specified column as a ClickHouseGeoPolygonValue. + * + * @param index + * @return + */ + ClickHouseGeoPolygonValue getGeoPolygon(int index); + + /** + * Returns the value of the specified column as a ClickHouseGeoMultiPolygonValue. + * + * @param index + * @return + */ + ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index); + + /** + * Returns the value of the specified column as a {@link List}. Suitable for reading Array columns of any type. + *

For nested arrays (e.g. {@code Array(Array(Int64))}), returns a {@code List>}. + * For nullable arrays (e.g. {@code Array(Nullable(Int32))}), list elements may be {@code null}.

+ * + * @param index - column index (1-based) + * @return list of values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the column is not an array type + */ + List getList(int index); + + /** + * Returns the value of the specified column as a {@code byte[]}. Suitable for 1D Array columns only. + * + * @param index - column index (1-based) + * @return array of bytes, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a byte array + */ + byte[] getByteArray(int index); + + /** + * Returns the value of the specified column as an {@code int[]}. Suitable for 1D Array columns only. + * + * @param index - column index (1-based) + * @return array of int values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to an int array + */ + int[] getIntArray(int index); + + /** + * Returns the value of the specified column as a {@code long[]}. Suitable for 1D Array columns only. + * + * @param index - column index (1-based) + * @return array of long values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a long array + */ + long[] getLongArray(int index); + + /** + * Returns the value of the specified column as a {@code float[]}. Suitable for 1D Array columns only. + * + * @param index - column index (1-based) + * @return array of float values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a float array + */ + float[] getFloatArray(int index); + + /** + * Returns the value of the specified column as a {@code double[]}. Suitable for 1D Array columns only. + * + * @param index - column index (1-based) + * @return array of double values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a double array + */ + double[] getDoubleArray(int index); + + /** + * Returns the value of the specified column as a {@code boolean[]}. Suitable for 1D Array columns only. + * + * @param index - column index (1-based) + * @return array of boolean values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a boolean array + */ + boolean[] getBooleanArray(int index); + + /** + * Returns the value of the specified column as a {@code short[]}. Suitable for 1D Array columns only. + * + * @param index - column index (1-based) + * @return array of short values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the value cannot be converted to a short array + */ + short[] getShortArray(int index); + + /** + * Returns the value of the specified column as a {@code String[]}. Suitable for 1D Array columns only. + * Cannot be used for none string element types. + * + * @param index - column index (1-based) + * @return array of string values, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the column is not an array type + */ + String[] getStringArray(int index); + + /** + * Returns the value of the specified column as an {@code Object[]}. Suitable for multidimensional Array columns. + * Nested arrays are recursively converted to {@code Object[]}. + * Note: result is not cached so avoid repetitive calls on same column. + * + * @param index - column index (1-based) + * @return array of objects, or {@code null} if the value is null + * @throws com.clickhouse.client.api.ClientException if the column is not an array type + */ + Object[] getObjectArray(int index); + + Object[] getTuple(int index); + + Object[] getTuple(String colName); + + byte getEnum8(String colName); + + byte getEnum8(int index); + + short getEnum16(String colName); + + short getEnum16(int index); + + LocalDate getLocalDate(String colName); + + LocalDate getLocalDate(int index); + + LocalTime getLocalTime(String colName); + + LocalTime getLocalTime(int index); + + LocalDateTime getLocalDateTime(String colName); + + LocalDateTime getLocalDateTime(int index); + + OffsetDateTime getOffsetDateTime(String colName); + + OffsetDateTime getOffsetDateTime(int index); + + TableSchema getSchema(); + + 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/ClickHouseTextFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseTextFormatReader.java new file mode 100644 index 000000000..c618d1bde --- /dev/null +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseTextFormatReader.java @@ -0,0 +1,46 @@ +package com.clickhouse.client.api.data_formats; + +/** + * Reader for ClickHouse text output formats (such as + * {@code JSONEachRow}). + * + *

Row navigation, schema access, and typed accessors are inherited from + * {@link ClickHouseFormatReader}; this interface specializes the contract for + * text-encoded result streams

+ * + *

Implementation of a reader may not support every accessor declared on + * {@link ClickHouseFormatReader}; unsupported accessors are expected to throw + * {@link UnsupportedOperationException}.

+ * + *

Format-specific accessor behavior

+ * + *

Text formats encode values as strings rather than as typed binary tokens, + * so a number of accessors inherited from {@link ClickHouseFormatReader} have + * looser or otherwise text-specific semantics:

+ * + *
    + *
  • {@code Enum8} / {@code Enum16}. ClickHouse always emits enum + * columns in text formats (including {@code JSONEachRow}) as their + * string label, never as the underlying numeric value. As a + * result, the inherited {@link ClickHouseFormatReader#getEnum8(String) + * getEnum8} / {@link ClickHouseFormatReader#getEnum16(String) getEnum16} + * accessors are not usable for real enum data and are expected to throw + * a {@link RuntimeException}. Callers should read enum columns with + * {@link ClickHouseFormatReader#getString(String) getString} instead.
  • + *
  • Temporal and other parse-on-read accessors + * ({@link ClickHouseFormatReader#getLocalDate(String) getLocalDate}, + * {@link ClickHouseFormatReader#getLocalTime(String) getLocalTime}, + * {@link ClickHouseFormatReader#getLocalDateTime(String) getLocalDateTime}, + * {@link ClickHouseFormatReader#getOffsetDateTime(String) getOffsetDateTime}, + * {@link ClickHouseFormatReader#getUUID(String) getUUID}) parse the value + * out of its textual representation, so they fail with a + * {@link RuntimeException} when the underlying value is not a valid + * literal for the requested type.
  • + *
  • Schema inference in text formats may be best-effort: when the + * reader cannot determine the original ClickHouse type from the wire + * content, the inferred {@code TableSchema} may report a coarser type + * than a binary reader would for the same column.
  • + *
+ */ +public interface ClickHouseTextFormatReader extends ClickHouseFormatReader { +} diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/GsonJsonParserFactory.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/GsonJsonParserFactory.java new file mode 100644 index 000000000..43ac72838 --- /dev/null +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/GsonJsonParserFactory.java @@ -0,0 +1,68 @@ +package com.clickhouse.client.api.data_formats; + +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.ToNumberPolicy; +import com.google.gson.reflect.TypeToken; +import com.google.gson.stream.JsonReader; +import com.google.gson.stream.JsonToken; + +import java.io.InputStream; +import java.io.InputStreamReader; +import java.lang.reflect.Type; +import java.nio.charset.StandardCharsets; +import java.util.Map; + +public class GsonJsonParserFactory implements JsonParserFactory { + private final Gson gson; + + private static final Type MAP_TYPE_TOKEN = new TypeToken>() {}.getType(); + + public GsonJsonParserFactory() { + GsonBuilder builder = new GsonBuilder(); + customize(builder); + builder.setLenient(); // JSONEachRow needs lenient reader for multiple root objects + this.gson = builder.create(); + } + + protected void customize(GsonBuilder builder) { + // JSONEachRow numbers may represent UInt64 or Decimal values, so avoid + // Gson's default Double materialization and preserve the original value. + builder.setObjectToNumberStrategy(ToNumberPolicy.BIG_DECIMAL); + } + + @Override + public JsonParser createJsonParser(InputStream in) { + return new JsonParserImpl(gson.newJsonReader(new InputStreamReader(in, StandardCharsets.UTF_8))); + } + + private class JsonParserImpl implements JsonParser { + + private final JsonReader reader; + + + + public JsonParserImpl(JsonReader jsonReader) { + this.reader = jsonReader; + } + + + @Override + public Map nextRow() throws Exception { + try { + if (reader.peek() == JsonToken.END_DOCUMENT) { + return null; + } + } catch (java.io.EOFException e) { + return null; + } + + return GsonJsonParserFactory.this.gson.fromJson(reader, GsonJsonParserFactory.MAP_TYPE_TOKEN); + } + + @Override + public void close() throws Exception { + reader.close(); + } + } +} diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/JSONEachRowFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/JSONEachRowFormatReader.java new file mode 100644 index 000000000..083a88d87 --- /dev/null +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/JSONEachRowFormatReader.java @@ -0,0 +1,645 @@ +package com.clickhouse.client.api.data_formats; + +import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.data_formats.internal.NumberConverter; +import com.clickhouse.client.api.internal.SchemaUtils; +import com.clickhouse.client.api.metadata.TableSchema; +import com.clickhouse.data.ClickHouseColumn; +import com.clickhouse.data.value.ClickHouseBitmap; +import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoPointValue; +import com.clickhouse.data.value.ClickHouseGeoPolygonValue; +import com.clickhouse.data.value.ClickHouseGeoRingValue; + +import 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.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZonedDateTime; +import java.time.temporal.TemporalAmount; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +public class JSONEachRowFormatReader implements ClickHouseTextFormatReader { + private final JsonParser parser; + private TableSchema schema; + private Map currentRow; + private Map nextRow; + private boolean hasNext; + + public JSONEachRowFormatReader(JsonParser parser) { + this.parser = parser; + try { + this.nextRow = parser.nextRow(); + this.hasNext = this.nextRow != null; + if (nextRow != null) { + // The schema is inferred from the keys of the first row in + // iteration order. The supplied JsonParser must therefore + // return an order-preserving Map (e.g. LinkedHashMap); see + // JsonParser#nextRow for the contract relied on here. + List columns = new ArrayList<>(); + for (String key : nextRow.keySet()) { + // For JSONEachRow we don't know the exact ClickHouse type, so we use a reasonable default. + // We can try to guess based on the value type in the first row. + columns.add(ClickHouseColumn.of(key, SchemaUtils.inferDataType(nextRow.get(key)), false)); + } + this.schema = new TableSchema(columns); + } else { + this.schema = new TableSchema(new ArrayList<>()); + } + } catch (Exception e) { + throw new RuntimeException("Failed to initialize JSON reader", e); + } + } + + @Override + public T readValue(int colIndex) { + return (T) currentRow.get(schema.columnIndexToName(colIndex)); + } + + @Override + public T readValue(String colName) { + return (T) currentRow.get(colName); + } + + @Override + public boolean hasValue(String colName) { + return currentRow.containsKey(colName) && currentRow.get(colName) != null; + } + + @Override + public boolean hasValue(int colIndex) { + return hasValue(schema.columnIndexToName(colIndex)); + } + + @Override + public boolean hasNext() { + return hasNext; + } + + @Override + public Map next() { + if (!hasNext) { + currentRow = null; + return null; + } + + currentRow = nextRow; + readNextRow(); + return currentRow; + } + + private void readNextRow() { + try { + nextRow = parser.nextRow(); + hasNext = nextRow != null; + } catch (Exception e) { + hasNext = false; + nextRow = null; + throw new RuntimeException("Failed to read next JSON row", e); + } + } + + @Override + public String getString(String colName) { + Object val = currentRow.get(colName); + return val == null ? null : val.toString(); + } + + @Override + public byte getByte(String colName) { + return ((Number) currentRow.get(colName)).byteValue(); + } + + @Override + public short getShort(String colName) { + return ((Number) currentRow.get(colName)).shortValue(); + } + + @Override + public int getInteger(String colName) { + return ((Number) currentRow.get(colName)).intValue(); + } + + @Override + public long getLong(String colName) { + return ((Number) currentRow.get(colName)).longValue(); + } + + @Override + public float getFloat(String colName) { + return ((Number) currentRow.get(colName)).floatValue(); + } + + @Override + public double getDouble(String colName) { + return ((Number) currentRow.get(colName)).doubleValue(); + } + + @Override + public boolean getBoolean(String colName) { + Object val = currentRow.get(colName); + if (val instanceof Boolean) { + return (Boolean) val; + } + if (val instanceof Number) { + // Match AbstractBinaryFormatReader (SerializerUtils.convertToBoolean): + // any non-zero integral value is true, zero is false. Fractional + // values keep the same behavior because they are truncated by + // longValue() before the zero check. + return ((Number) val).longValue() != 0; + } + if (val == null) { + throw new ClientException("Column '" + colName + "' has null value and cannot be converted to boolean"); + } + throw new ClientException("Cannot convert value of type " + val.getClass().getName() + + " in column '" + colName + "' to boolean"); + } + + @Override + public BigInteger getBigInteger(String colName) { + Object val = currentRow.get(colName); + if (val == null) return null; + if (val instanceof BigInteger) return (BigInteger) val; + return new BigDecimal(val.toString()).toBigInteger(); + } + + @Override + public BigDecimal getBigDecimal(String colName) { + Object val = currentRow.get(colName); + if (val == null) return null; + if (val instanceof BigDecimal) return (BigDecimal) val; + return new BigDecimal(val.toString()); + } + + @Override + public Instant getInstant(String colName) { + throw new UnsupportedOperationException(); + } + + @Override + public ZonedDateTime getZonedDateTime(String colName) { + throw new UnsupportedOperationException(); + } + + @Override + public Duration getDuration(String colName) { + throw new UnsupportedOperationException(); + } + + @Override + public Inet4Address getInet4Address(String colName) { + throw new UnsupportedOperationException(); + } + + @Override + public Inet6Address getInet6Address(String colName) { + throw new UnsupportedOperationException(); + } + + @Override + public UUID getUUID(String colName) { + Object val = currentRow.get(colName); + return val == null ? null : UUID.fromString(val.toString()); + } + + @Override + public ClickHouseGeoPointValue getGeoPoint(String colName) { + throw new UnsupportedOperationException(); + } + + @Override + public ClickHouseGeoRingValue getGeoRing(String colName) { + throw new UnsupportedOperationException(); + } + + @Override + public ClickHouseGeoPolygonValue getGeoPolygon(String colName) { + throw new UnsupportedOperationException(); + } + + @Override + public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) { + throw new UnsupportedOperationException(); + } + + @Override + public List getList(String colName) { + Object val = currentRow.get(colName); + if (val == null) { + return null; + } + if (!(val instanceof List)) { + throw new ClientException("Column '" + colName + "' is not of array type (actual: " + + val.getClass().getName() + ")"); + } + return (List) val; + } + + @Override + public byte[] getByteArray(String colName) { + return getPrimitiveArray(colName, byte.class); + } + + @Override + public int[] getIntArray(String colName) { + return getPrimitiveArray(colName, int.class); + } + + @Override + public long[] getLongArray(String colName) { + return getPrimitiveArray(colName, long.class); + } + + @Override + public float[] getFloatArray(String colName) { + return getPrimitiveArray(colName, float.class); + } + + @Override + public double[] getDoubleArray(String colName) { + return getPrimitiveArray(colName, double.class); + } + + @Override + public boolean[] getBooleanArray(String colName) { + return getPrimitiveArray(colName, boolean.class); + } + + @Override + public short[] getShortArray(String colName) { + return getPrimitiveArray(colName, short.class); + } + + @Override + public String[] getStringArray(String colName) { + List list = asArrayList(colName); + if (list == null) { + return null; + } + String[] out = new String[list.size()]; + for (int i = 0; i < list.size(); i++) { + Object el = list.get(i); + out[i] = el == null ? null : el.toString(); + } + return out; + } + + @Override + public Object[] getObjectArray(String colName) { + List list = asArrayList(colName); + return list == null ? null : list.toArray(new Object[0]); + } + + /** + * Returns the value of the given column as a {@code List}, or {@code null} + * if the value is missing. Throws {@link ClientException} when the column + * exists but is not an array. + */ + private List asArrayList(String colName) { + Object val = currentRow.get(colName); + if (val == null) { + return null; + } + if (!(val instanceof List)) { + throw new ClientException("Column '" + colName + "' is not of array type (actual: " + + val.getClass().getName() + ")"); + } + return (List) val; + } + + @SuppressWarnings("unchecked") + private T getPrimitiveArray(String colName, Class componentType) { + List list = asArrayList(colName); + if (list == null) { + return null; + } + try { + Object array = Array.newInstance(componentType, list.size()); + for (int i = 0; i < list.size(); i++) { + Object el = list.get(i); + if (el == null) { + throw new ClientException("Column '" + colName + + "' contains a null element which cannot fit into an array of primitive " + + componentType.getName()); + } + Array.set(array, i, coerceToComponent(el, componentType)); + } + return (T) array; + } catch (ClassCastException | IllegalArgumentException e) { + throw new ClientException("Value of column '" + colName + + "' cannot be converted to an array of " + componentType.getName(), e); + } + } + + /** + * Coerces a parsed JSON element to a boxed primitive type. JSON parsers + * may materialize numeric array elements as different boxed types + * (e.g. {@code Integer}, {@code Long}, {@code Double}, {@code BigDecimal}), + * so element-level conversion is necessary before populating a typed + * primitive array. The {@code componentType} is always one of the eight + * Java primitives passed by {@link #getPrimitiveArray}; unsupported + * component types are rejected explicitly to keep the helper total. + */ + private static Object coerceToComponent(Object value, Class componentType) { + if (componentType == byte.class) { + return NumberConverter.toByte(value); + } + if (componentType == short.class) { + return NumberConverter.toShort(value); + } + if (componentType == int.class) { + return NumberConverter.toInt(value); + } + if (componentType == long.class) { + return NumberConverter.toLong(value); + } + if (componentType == float.class) { + return NumberConverter.toFloat(value); + } + if (componentType == double.class) { + return NumberConverter.toDouble(value); + } + if (componentType == boolean.class) { + if (value instanceof Boolean) { + return value; + } + if (value instanceof Number) { + return ((Number) value).longValue() != 0; + } + throw new IllegalArgumentException( + "Cannot convert " + value.getClass().getName() + " to boolean array element"); + } + throw new IllegalArgumentException("Unsupported component type: " + componentType.getName()); + } + + @Override + public String getString(int index) { + return getString(schema.columnIndexToName(index)); + } + + @Override + public byte getByte(int index) { + return getByte(schema.columnIndexToName(index)); + } + + @Override + public short getShort(int index) { + return getShort(schema.columnIndexToName(index)); + } + + @Override + public int getInteger(int index) { + return getInteger(schema.columnIndexToName(index)); + } + + @Override + public long getLong(int index) { + return getLong(schema.columnIndexToName(index)); + } + + @Override + public float getFloat(int index) { + return getFloat(schema.columnIndexToName(index)); + } + + @Override + public double getDouble(int index) { + return getDouble(schema.columnIndexToName(index)); + } + + @Override + public boolean getBoolean(int index) { + return getBoolean(schema.columnIndexToName(index)); + } + + @Override + public BigInteger getBigInteger(int index) { + return getBigInteger(schema.columnIndexToName(index)); + } + + @Override + public BigDecimal getBigDecimal(int index) { + return getBigDecimal(schema.columnIndexToName(index)); + } + + @Override + public Instant getInstant(int index) { + return getInstant(schema.columnIndexToName(index)); + } + + @Override + public ZonedDateTime getZonedDateTime(int index) { + return getZonedDateTime(schema.columnIndexToName(index)); + } + + @Override + public Duration getDuration(int index) { + return getDuration(schema.columnIndexToName(index)); + } + + @Override + public Inet4Address getInet4Address(int index) { + return getInet4Address(schema.columnIndexToName(index)); + } + + @Override + public Inet6Address getInet6Address(int index) { + return getInet6Address(schema.columnIndexToName(index)); + } + + @Override + public UUID getUUID(int index) { + return getUUID(schema.columnIndexToName(index)); + } + + @Override + public ClickHouseGeoPointValue getGeoPoint(int index) { + return getGeoPoint(schema.columnIndexToName(index)); + } + + @Override + public ClickHouseGeoRingValue getGeoRing(int index) { + return getGeoRing(schema.columnIndexToName(index)); + } + + @Override + public ClickHouseGeoPolygonValue getGeoPolygon(int index) { + return getGeoPolygon(schema.columnIndexToName(index)); + } + + @Override + public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) { + return getGeoMultiPolygon(schema.columnIndexToName(index)); + } + + @Override + public List getList(int index) { + return getList(schema.columnIndexToName(index)); + } + + @Override + public byte[] getByteArray(int index) { + return getByteArray(schema.columnIndexToName(index)); + } + + @Override + public int[] getIntArray(int index) { + return getIntArray(schema.columnIndexToName(index)); + } + + @Override + public long[] getLongArray(int index) { + return getLongArray(schema.columnIndexToName(index)); + } + + @Override + public float[] getFloatArray(int index) { + return getFloatArray(schema.columnIndexToName(index)); + } + + @Override + public double[] getDoubleArray(int index) { + return getDoubleArray(schema.columnIndexToName(index)); + } + + @Override + public boolean[] getBooleanArray(int index) { + return getBooleanArray(schema.columnIndexToName(index)); + } + + @Override + public short[] getShortArray(int index) { + return getShortArray(schema.columnIndexToName(index)); + } + + @Override + public String[] getStringArray(int index) { + return getStringArray(schema.columnIndexToName(index)); + } + + @Override + public Object[] getObjectArray(int index) { + return getObjectArray(schema.columnIndexToName(index)); + } + + @Override + public Object[] getTuple(int index) { + return getTuple(schema.columnIndexToName(index)); + } + + @Override + public Object[] getTuple(String colName) { + Object value = currentRow.get(colName); + if (value == null) { + return null; + } + if (value instanceof List) { + return ((List) value).toArray(new Object[0]); + } + return (Object[]) value; + } + + @Override + public byte getEnum8(String colName) { + return getByte(colName); + } + + @Override + public byte getEnum8(int index) { + return getByte(index); + } + + @Override + public short getEnum16(String colName) { + return getShort(colName); + } + + @Override + public short getEnum16(int index) { + return getShort(index); + } + + @Override + public LocalDate getLocalDate(String colName) { + Object val = currentRow.get(colName); + return val == null ? null : LocalDate.parse(val.toString()); + } + + @Override + public LocalDate getLocalDate(int index) { + return getLocalDate(schema.columnIndexToName(index)); + } + + @Override + public LocalTime getLocalTime(String colName) { + Object val = currentRow.get(colName); + return val == null ? null : LocalTime.parse(val.toString()); + } + + @Override + public LocalTime getLocalTime(int index) { + return getLocalTime(schema.columnIndexToName(index)); + } + + @Override + public LocalDateTime getLocalDateTime(String colName) { + Object val = currentRow.get(colName); + return val == null ? null : LocalDateTime.parse(val.toString()); + } + + @Override + public LocalDateTime getLocalDateTime(int index) { + return getLocalDateTime(schema.columnIndexToName(index)); + } + + @Override + public OffsetDateTime getOffsetDateTime(String colName) { + Object val = currentRow.get(colName); + return val == null ? null : OffsetDateTime.parse(val.toString()); + } + + @Override + public OffsetDateTime getOffsetDateTime(int index) { + return getOffsetDateTime(schema.columnIndexToName(index)); + } + + @Override + public TableSchema getSchema() { + return schema; + } + + @Override + public ClickHouseBitmap getClickHouseBitmap(String colName) { + throw new UnsupportedOperationException(); + } + + @Override + public ClickHouseBitmap getClickHouseBitmap(int index) { + throw new UnsupportedOperationException(); + } + + @Override + public TemporalAmount getTemporalAmount(int index) { + throw new UnsupportedOperationException(); + } + + @Override + public TemporalAmount getTemporalAmount(String colName) { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws Exception { + parser.close(); + } +} diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/JacksonJsonParserFactory.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/JacksonJsonParserFactory.java new file mode 100644 index 000000000..0730be9c0 --- /dev/null +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/JacksonJsonParserFactory.java @@ -0,0 +1,51 @@ +package com.clickhouse.client.api.data_formats; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; + +public class JacksonJsonParserFactory implements JsonParserFactory { + private final ObjectMapper mapper; + + public JacksonJsonParserFactory() { + this.mapper = createMapper(); + } + + protected ObjectMapper createMapper() { + // override this method to customize object mapper + return new ObjectMapper(); + } + + @Override + public JsonParser createJsonParser(InputStream in) throws IOException { + return new JsonParserImpl(mapper.createParser(in)); + } + + private class JsonParserImpl implements JsonParser { + + private final com.fasterxml.jackson.core.JsonParser parser; + + public JsonParserImpl(com.fasterxml.jackson.core.JsonParser parser) { + this.parser = parser; + } + + @Override + public Map nextRow() throws Exception { + // Jackson's streaming parser skips whitespace (including the newlines that + // separate JSONEachRow objects), so reaching EOF is the only reason + // nextToken() returns null. Any non-START_OBJECT token here would indicate + // malformed input and is reported by mapper.readValue(...). + if (parser.nextToken() == null) { + return null; + } + return mapper.readValue(parser, Map.class); + } + + @Override + public void close() throws Exception { + parser.close(); + } + } +} diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/JsonParser.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/JsonParser.java new file mode 100644 index 000000000..564f89192 --- /dev/null +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/JsonParser.java @@ -0,0 +1,41 @@ +package com.clickhouse.client.api.data_formats; + +import java.util.Map; + +/** + * Interface for JSON row processors. + * + *

Implementations are responsible for reading one JSON object per call + * from the underlying input stream and returning its fields as a + * {@link Map} keyed by column name.

+ * + *

Iteration-order contract

+ * + *

The reader infers its column schema from the keys of the first + * row, in iteration order, and uses that order to resolve 1-based column + * indexes for every subsequent row. Implementations must therefore + * return a {@link Map} that preserves the insertion order of the JSON keys + * as they appeared on the wire (e.g. {@link java.util.LinkedHashMap}); a + * hash-ordered {@link java.util.HashMap} is not acceptable because it + * destabilises index-based access.

+ * + *

The bundled {@link JacksonJsonParserFactory} (which produces + * {@link java.util.LinkedHashMap} via Jackson's {@code Map.class} default) + * and {@link GsonJsonParserFactory} (which produces {@code LinkedTreeMap}) + * both satisfy this contract. Custom implementations must take care not to + * regress it.

+ */ +public interface JsonParser extends AutoCloseable { + + + /** + * Reads next row from the input stream. + * + *

The returned map must preserve the insertion order of the JSON + * keys as defined by the {@linkplain JsonParser interface contract}.

+ * + * @return map of column names to values, or null if no more rows + * @throws Exception if an error occurs during parsing + */ + Map nextRow() throws Exception; +} diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/JsonParserFactory.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/JsonParserFactory.java new file mode 100644 index 000000000..94bd3e5c9 --- /dev/null +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/JsonParserFactory.java @@ -0,0 +1,17 @@ +package com.clickhouse.client.api.data_formats; + +import java.io.IOException; +import java.io.InputStream; + +public interface JsonParserFactory { + + + /** + * Implementation should create only instance of actual JSON parser. + * This method is called for each request and should avoid long initialization or + * create big objects + * @param in - stream of bytes to parse as JSON + * @return instance of {@link JsonParser} + */ + JsonParser createJsonParser(InputStream in) throws IOException; +} diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/SchemaUtils.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/SchemaUtils.java new file mode 100644 index 000000000..ed0ce67b6 --- /dev/null +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/SchemaUtils.java @@ -0,0 +1,87 @@ +package com.clickhouse.client.api.internal; + +import com.clickhouse.data.ClickHouseDataType; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public final class SchemaUtils { + private static final ClickHouseDataType DEFAULT_DATA_TYPE = ClickHouseDataType.String; + + private static final List DATA_TYPE_PRIORITY = ImmutableList.of( + ClickHouseDataType.Bool, + ClickHouseDataType.Int8, + ClickHouseDataType.Int16, + ClickHouseDataType.Int32, + ClickHouseDataType.Int64, + ClickHouseDataType.Int256, + ClickHouseDataType.Float32, + ClickHouseDataType.Float64, + ClickHouseDataType.Decimal, + ClickHouseDataType.String, + ClickHouseDataType.UUID, + ClickHouseDataType.IPv4, + ClickHouseDataType.IPv6, + ClickHouseDataType.DateTime64, + ClickHouseDataType.Date, + ClickHouseDataType.IntervalNanosecond, + ClickHouseDataType.IntervalDay, + ClickHouseDataType.Time64, + ClickHouseDataType.Point, + ClickHouseDataType.Ring, + ClickHouseDataType.Polygon, + ClickHouseDataType.MultiPolygon, + ClickHouseDataType.Tuple, + ClickHouseDataType.Geometry + ); + + private static final Map, ClickHouseDataType> CLASS_TO_DATA_TYPE = buildClassToDataTypeMap(); + + private SchemaUtils() { + } + + public static ClickHouseDataType inferDataType(Object value) { + if (value == null) { + return DEFAULT_DATA_TYPE; + } + + // JSONEachRow has no type metadata, so structural values only infer the + // top-level family; nested key/value/element types remain best-effort. + if (value instanceof Map) { + return ClickHouseDataType.Map; + } + + Class valueClass = ClickHouseDataType.toObjectType(value.getClass()); + if (value instanceof List || valueClass.isArray()) { + return ClickHouseDataType.Array; + } + + ClickHouseDataType dataType = CLASS_TO_DATA_TYPE.get(valueClass); + return dataType == null ? DEFAULT_DATA_TYPE : dataType; + } + + private static Map, ClickHouseDataType> buildClassToDataTypeMap() { + Map, ClickHouseDataType> map = new LinkedHashMap<>(); + + for (ClickHouseDataType dataType : DATA_TYPE_PRIORITY) { + addTypeMappings(map, dataType); + } + + return ImmutableMap.copyOf(map); + } + + private static void addTypeMappings(Map, ClickHouseDataType> map, ClickHouseDataType dataType) { + Set> javaClasses = ClickHouseDataType.DATA_TYPE_TO_CLASS.get(dataType); + if (javaClasses == null) { + return; + } + + for (Class javaClass : javaClasses) { + map.putIfAbsent(ClickHouseDataType.toObjectType(javaClass), dataType); + } + } +} diff --git a/client-v2/src/test/java/com/clickhouse/client/ClientTests.java b/client-v2/src/test/java/com/clickhouse/client/ClientTests.java index eaa675349..a308d8949 100644 --- a/client-v2/src/test/java/com/clickhouse/client/ClientTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/ClientTests.java @@ -332,7 +332,7 @@ public void testDefaultSettings() { Assert.assertEquals(config.get(p.getKey()), p.getDefaultValue(), "Default value doesn't match"); } } - Assert.assertEquals(config.size(), 34); // to check everything is set. Increment when new added. + Assert.assertEquals(config.size(), 35); // to check everything is set. Increment when new added. } try (Client client = new Client.Builder() @@ -365,7 +365,7 @@ public void testDefaultSettings() { .setSocketSndbuf(100000) .build()) { Map config = client.getConfiguration(); - Assert.assertEquals(config.size(), 35); // to check everything is set. Increment when new added. + Assert.assertEquals(config.size(), 36); // to check everything is set. Increment when new added. Assert.assertEquals(config.get(ClientConfigProperties.DATABASE.getKey()), "mydb"); Assert.assertEquals(config.get(ClientConfigProperties.MAX_EXECUTION_TIME.getKey()), "10"); Assert.assertEquals(config.get(ClientConfigProperties.COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE.getKey()), "300000"); @@ -432,7 +432,7 @@ public void testWithOldDefaults() { Assert.assertEquals(config.get(p.getKey()), p.getDefaultValue(), "Default value doesn't match"); } } - Assert.assertEquals(config.size(), 34); // to check everything is set. Increment when new added. + Assert.assertEquals(config.size(), 35); // to check everything is set. Increment when new added. } } diff --git a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/AbstractJSONEachRowFormatReaderTests.java b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/AbstractJSONEachRowFormatReaderTests.java new file mode 100644 index 000000000..e95920d06 --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/AbstractJSONEachRowFormatReaderTests.java @@ -0,0 +1,1261 @@ +package com.clickhouse.client.api.data_formats; + +import com.clickhouse.client.BaseIntegrationTest; +import com.clickhouse.client.ClickHouseNode; +import com.clickhouse.client.ClickHouseProtocol; +import com.clickhouse.client.ClickHouseServerForTest; +import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.ClientConfigProperties; +import com.clickhouse.client.api.enums.Protocol; +import com.clickhouse.client.api.query.GenericRecord; +import com.clickhouse.client.api.query.QueryResponse; +import com.clickhouse.client.api.query.QuerySettings; +import com.clickhouse.data.ClickHouseDataType; +import com.clickhouse.data.ClickHouseFormat; +import com.clickhouse.data.ClickHouseVersion; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.charset.StandardCharsets; +import java.time.LocalDate; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import java.util.Random; +import java.util.UUID; +import java.util.function.BiFunction; +import java.util.function.BiPredicate; +import java.util.function.Consumer; + +public abstract class AbstractJSONEachRowFormatReaderTests extends BaseIntegrationTest { + + /** Number of rows generated per primitive column. */ + private static final int ROW_COUNT = 5; + + /** Fixed seed so generated random values are stable across runs and parser implementations. */ + private static final long RANDOM_SEED = 0xC0FFEEL; + + /** Shared list of primitive cases, populated once per JVM and reused by every test class. */ + private static final List PRIMITIVE_CASES = + buildPrimitiveCases(new Random(RANDOM_SEED)); + + protected Client client; + + private String primitivesTable; + + @BeforeClass(groups = {"integration"}) + public void setUpPrimitivesTable() throws Exception { + primitivesTable = "test_json_each_row_primitives_" + + getClass().getSimpleName().toLowerCase(Locale.ROOT); + + ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); + try (Client setupClient = new Client.Builder() + .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), isCloud()) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .build()) { + + setupClient.execute("DROP TABLE IF EXISTS " + primitivesTable).get().close(); + + StringBuilder create = new StringBuilder("CREATE TABLE ") + .append(primitivesTable) + .append(" (id UInt32"); + for (PrimitiveTypeCase c : PRIMITIVE_CASES) { + create.append(", ").append(c.columnName).append(' ').append(c.chType); + } + create.append(") ENGINE = MergeTree ORDER BY id"); + setupClient.execute(create.toString()).get().close(); + + StringBuilder insert = new StringBuilder("INSERT INTO ") + .append(primitivesTable).append(" VALUES "); + for (int row = 0; row < ROW_COUNT; row++) { + if (row > 0) { + insert.append(", "); + } + insert.append('(').append(row); + for (PrimitiveTypeCase c : PRIMITIVE_CASES) { + insert.append(", ").append(c.sqlLiterals.get(row)); + } + insert.append(')'); + } + setupClient.execute(insert.toString()).get().close(); + } + } + + @AfterClass(groups = {"integration"}) + public void tearDownPrimitivesTable() throws Exception { + if (primitivesTable == null) { + return; + } + ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); + try (Client teardownClient = new Client.Builder() + .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), isCloud()) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .build()) { + teardownClient.execute("DROP TABLE IF EXISTS " + primitivesTable).get().close(); + } + } + + @BeforeMethod(groups = {"integration"}) + public void setUp() { + ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); + client = new Client.Builder() + .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), isCloud()) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .build(); + } + + @AfterMethod(groups = {"integration"}) + public void tearDown() { + if (client != null) { + client.close(); + } + } + + private QuerySettings newJsonEachRowSettings() { + return new QuerySettings() + .setFormat(ClickHouseFormat.JSONEachRow); + } + + /** + * Settings used by the primitive accessor tests. Integer and floating-point values + * are returned unquoted so they materialise as {@code Number} instances (whose + * {@code longValue}/{@code doubleValue} calls match the typed accessors). + * Decimal values are kept quoted (the ClickHouse default) so that JSON parsers + * that materialise unquoted JSON numbers as {@code Double} (e.g. Jackson with + * default settings) do not lose precision on large {@code Decimal} values. + */ + private QuerySettings newJsonEachRowSettingsForPrimitives() { + return newJsonEachRowSettings() + .serverSetting("output_format_json_quote_64bit_integers", "0") + .serverSetting("output_format_json_quote_64bit_floats", "0") + .serverSetting("output_format_json_quote_decimals", "1"); + } + + protected abstract ClickHouseTextFormatReader createReader(QueryResponse response) throws IOException; + + /** + * Builds a reader directly over the provided JSONEachRow byte stream, bypassing + * the server. This is used to exercise error paths (such as corrupted input) + * deterministically across both parser factories. + */ + protected abstract ClickHouseTextFormatReader createReader(InputStream input) throws IOException; + + // ------------------------------------------------------------------ + // Parameterized primitive value tests + // ------------------------------------------------------------------ + + @DataProvider(name = "primitiveTypeCases") + public Object[][] primitiveTypeCases() { + Object[][] rows = new Object[PRIMITIVE_CASES.size()][1]; + for (int i = 0; i < PRIMITIVE_CASES.size(); i++) { + rows[i][0] = PRIMITIVE_CASES.get(i); + } + return rows; + } + + /** + * Runs the same assertions for every primitive type by reading the column populated + * in {@link #setUpPrimitivesTable()}. For each row it asserts that the value can be + * read via the type-appropriate accessor (both by name and by index) and that a set + * of accessors that cannot convert the value throw an exception. + */ + @Test(groups = {"integration"}, dataProvider = "primitiveTypeCases") + public void testPrimitiveTypeAccessors(PrimitiveTypeCase tc) throws Exception { + String sql = "SELECT " + tc.columnName + " FROM " + primitivesTable + " ORDER BY id"; + + try (QueryResponse response = + client.query(sql, newJsonEachRowSettingsForPrimitives()).get(); + ClickHouseTextFormatReader reader = createReader(response)) { + + for (int row = 0; row < ROW_COUNT; row++) { + Assert.assertTrue(reader.hasNext(), + "missing row " + row + " for " + tc); + Assert.assertNotNull(reader.next(), + "null row " + row + " for " + tc); + + Object expected = tc.expectedValues.get(row); + + Object actualByName = tc.readByName.apply(reader, tc.columnName); + Assert.assertTrue(tc.equality.test(actualByName, expected), + "row " + row + " by name for " + tc + + ": expected=" + expected + ", actual=" + actualByName); + + Object actualByIndex = tc.readByIndex.apply(reader, 1); + Assert.assertTrue(tc.equality.test(actualByIndex, expected), + "row " + row + " by index for " + tc + + ": expected=" + expected + ", actual=" + actualByIndex); + } + + Assert.assertFalse(reader.hasNext(), "extra rows for " + tc); + Assert.assertNull(reader.next(), "extra row payload for " + tc); + } + + try (QueryResponse response = + client.query(sql, newJsonEachRowSettingsForPrimitives()).get(); + ClickHouseTextFormatReader reader = createReader(response)) { + + Assert.assertNotNull(reader.next(), "row needed for incompatibility checks: " + tc); + + for (IncompatibleAccessor accessor : tc.incompatibleAccessors) { + assertAccessorThrows(reader, accessor.byName, + accessor.name + " by name on " + tc); + assertAccessorThrows(reader, accessor.byIndex, + accessor.name + " by index on " + tc); + } + } + } + + private static void assertAccessorThrows(ClickHouseTextFormatReader reader, + Consumer call, + String context) { + try { + call.accept(reader); + Assert.fail("Expected exception when invoking " + context); + } catch (RuntimeException expected) { + // Any RuntimeException is acceptable - DateTimeParseException, + // NumberFormatException, ClassCastException, IllegalArgumentException + // and UnsupportedOperationException are all valid signals that the + // accessor cannot convert the stored value. + } + } + + // ------------------------------------------------------------------ + // Remaining non-value-focused tests + // ------------------------------------------------------------------ + + @Test(groups = {"integration"}) + public void testSchemaInference() throws Exception { + // Numeric inference depends on parser materialization, so this test checks + // that numerics do not collapse to String and stable scalar types still map. + // The server default for `output_format_json_quote_64bit_integers` flipped from + // `1` to `0` in ClickHouse 25.8 (PR #74079, backward-incompatible change). With + // `newJsonEachRowSettings()` the client does not override the server default, so + // what the reader infers for `Int64` reflects what the server actually emits: + // * pre-25.8 self-hosted: Int64 arrives as a quoted JSON string -> inferred as String + // * 25.8+ self-hosted: Int64 arrives as a JSON number -> inferred as a numeric + // (parser-specific) type + // * cloud : the default may be overridden by the cloud profile, so the + // live value of `output_format_json_quote_64bit_integers` is + // queried from `system.settings` to decide which contract holds. + // The cross-version contract for forcing numeric output regardless of server + // default is exercised by testSchemaInferenceWithNumberQuotingDisabled below. + String sql = "SELECT toInt64(42) as col_int, toFloat64(3.14) as col_float, " + + "true as col_bool, 'val' as col_str"; + + boolean int64InferredAsString = isInt64EmittedAsQuotedJsonString(); + + try (QueryResponse response = client.query(sql, newJsonEachRowSettings()).get(); + ClickHouseTextFormatReader reader = createReader(response)) { + + Assert.assertNotNull(reader.getSchema()); + Assert.assertEquals(reader.getSchema().getColumns().size(), 4); + + ClickHouseDataType colIntType = reader.getSchema().getColumnByIndex(1).getDataType(); + if (int64InferredAsString) { + Assert.assertEquals(colIntType, ClickHouseDataType.String, + "Int64 must be inferred as String when the server emits it as a quoted JSON string"); + } else { + Assert.assertNotEquals(colIntType, ClickHouseDataType.String, + "Int64 must not collapse to String when the server emits it as an unquoted JSON number"); + } + Assert.assertNotEquals(reader.getSchema().getColumnByIndex(2).getDataType(), ClickHouseDataType.String); + Assert.assertEquals(reader.getSchema().getColumnByIndex(3).getDataType(), ClickHouseDataType.Bool); + Assert.assertEquals(reader.getSchema().getColumnByIndex(4).getDataType(), ClickHouseDataType.String); + } + } + + /** + * Returns whether the connected server currently emits {@code Int64} values as quoted JSON + * strings (rather than as unquoted JSON numbers) when {@code newJsonEachRowSettings()} is + * used as-is, without overriding {@code output_format_json_quote_64bit_integers}. + * + *

Self-hosted servers follow the well-known default flip in 25.8 (PR #74079). Cloud + * deployments may carry a profile-level override of the same setting, so we read the live + * value from {@code system.settings} instead of inferring it from the version string.

+ */ + private boolean isInt64EmittedAsQuotedJsonString() { + if (isCloud()) { + List records = client.queryAll( + "SELECT value FROM system.settings WHERE name = 'output_format_json_quote_64bit_integers'"); + return !records.isEmpty() && "1".equals(records.get(0).getString(1)); + } + return !isVersionMatch("[25.8,)"); + } + + /** + * Locks in the cross-version contract of {@link ClientConfigProperties#JSON_DISABLE_NUMBER_QUOTING}: + * when the option is enabled, the reader must always see {@code Int64}, {@code Float64} and + * {@code Decimal} as plain JSON numbers and infer non-{@code String} types, regardless of + * whether the server defaults to quoting 64-bit integers (pre-25.8) or not (25.8+). + */ + @Test(groups = {"integration"}) + public void testSchemaInferenceWithNumberQuotingDisabled() throws Exception { + String sql = "SELECT toInt64(42) as col_int, toFloat64(3.14) as col_float, " + + "toDecimal64(1.5, 2) as col_dec"; + + QuerySettings settings = newJsonEachRowSettings() + .setOption(ClientConfigProperties.JSON_DISABLE_NUMBER_QUOTING.getKey(), true); + + try (QueryResponse response = client.query(sql, settings).get(); + ClickHouseTextFormatReader reader = createReader(response)) { + + Assert.assertNotNull(reader.getSchema()); + Assert.assertEquals(reader.getSchema().getColumns().size(), 3); + + Assert.assertNotEquals(reader.getSchema().getColumnByIndex(1).getDataType(), + ClickHouseDataType.String, + "Int64 must not be inferred as String when JSON_DISABLE_NUMBER_QUOTING is on"); + Assert.assertNotEquals(reader.getSchema().getColumnByIndex(2).getDataType(), + ClickHouseDataType.String, + "Float64 must not be inferred as String when JSON_DISABLE_NUMBER_QUOTING is on"); + Assert.assertNotEquals(reader.getSchema().getColumnByIndex(3).getDataType(), + ClickHouseDataType.String, + "Decimal must not be inferred as String when JSON_DISABLE_NUMBER_QUOTING is on"); + + Assert.assertNotNull(reader.next()); + Assert.assertEquals(reader.getLong("col_int"), 42L); + Assert.assertEquals(reader.getDouble("col_float"), 3.14d, 1e-9); + Assert.assertEquals(reader.getBigDecimal("col_dec").compareTo(new BigDecimal("1.50")), 0); + } + } + + private 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 testEmptyData() throws Exception { + String sql = "SELECT * FROM remote('127.0.0.1', system.one) WHERE dummy > 1"; + + try (QueryResponse response = client.query(sql, newJsonEachRowSettings()).get(); + ClickHouseTextFormatReader reader = createReader(response)) { + + Assert.assertFalse(reader.hasNext()); + Assert.assertNull(reader.next()); + Assert.assertEquals(reader.getSchema().getColumns().size(), 0); + } + } + + @Test(groups = {"integration"}) + public void testReadValueAndHasValue() throws Exception { + String sql = "SELECT 7 as id, 'abc' as name, CAST(NULL AS Nullable(String)) as missing"; + + try (QueryResponse response = client.query(sql, newJsonEachRowSettings()).get(); + ClickHouseTextFormatReader reader = createReader(response)) { + + reader.next(); + + Number id = reader.readValue("id"); + Assert.assertNotNull(id); + Assert.assertEquals(id.intValue(), 7); + Assert.assertEquals((String) reader.readValue(2), "abc"); + + Assert.assertTrue(reader.hasValue("id")); + Assert.assertTrue(reader.hasValue(2)); + Assert.assertFalse(reader.hasValue("missing")); + Assert.assertFalse(reader.hasValue(3)); + Assert.assertFalse(reader.hasValue("not_a_column")); + } + } + + @Test(groups = {"integration"}) + public void testListAccessor() throws Exception { + String sql = "SELECT [1, 2, 3] as arr"; + + try (QueryResponse response = client.query(sql, newJsonEachRowSettings()).get(); + ClickHouseTextFormatReader reader = createReader(response)) { + + reader.next(); + + List values = reader.getList("arr"); + Assert.assertNotNull(values); + Assert.assertEquals(values.size(), 3); + Assert.assertEquals(values.get(0).intValue(), 1); + Assert.assertEquals(values.get(1).intValue(), 2); + Assert.assertEquals(values.get(2).intValue(), 3); + + List byIndex = reader.getList(1); + Assert.assertNotNull(byIndex); + Assert.assertEquals(byIndex.size(), 3); + } + } + + @Test(groups = {"integration"}, expectedExceptions = IllegalArgumentException.class) + public void testNewBinaryFormatReaderRejectsJsonEachRow() throws Exception { + String sql = "SELECT 1 as id"; + + try (QueryResponse response = client.query(sql, newJsonEachRowSettings()).get()) { + client.newBinaryFormatReader(response); + } + } + + // ------------------------------------------------------------------ + // Corrupted-stream coverage for the readNextRow error path + // ------------------------------------------------------------------ + + /** + * Builds a reader directly over a hand-crafted JSONEachRow stream that + * contains a valid first object followed by malformed bytes. The first + * call to {@code next()} buffered row #1 in the constructor and then + * tries to read row #2, which is malformed; the reader must surface a + * {@link RuntimeException} and refuse to advance any further. + * + *

This is structured as a unit-style test so it does not depend on the + * server, but it stays in the {@code integration} group alongside the + * other reader tests because it is parser-factory specific.

+ */ + @Test(groups = {"integration"}) + public void testCorruptedStreamFailsReadNextRow() throws Exception { + // The third "row" deliberately contains an unterminated string after + // a partial object so that lenient parsers cannot quietly accept it. + // The newline separation matches real JSONEachRow framing. + String body = "{\"id\":1,\"name\":\"first\"}\n" + + "{\"id\":2,\"name\":\"second\"}\n" + + "{\"id\":3,\"name\":\"unterminated"; + + try (InputStream input = new ByteArrayInputStream(body.getBytes(StandardCharsets.UTF_8)); + ClickHouseTextFormatReader reader = createReader(input)) { + + Assert.assertTrue(reader.hasNext(), "first row should be buffered"); + + Assert.assertNotNull(reader.next(), "row 1 should be readable"); + Assert.assertEquals(reader.getString("name"), "first"); + + try { + reader.next(); + Assert.fail("Expected RuntimeException reading malformed row"); + } catch (RuntimeException expected) { + // any RuntimeException is acceptable - both Jackson and Gson + // surface different concrete exception types here, but the + // reader wraps them into a RuntimeException. + } + + Assert.assertFalse(reader.hasNext(), + "reader must report end-of-stream after a parse failure"); + Assert.assertNull(reader.next(), + "next() must return null after a parse failure, not retry the stream"); + } + } + + // ------------------------------------------------------------------ + // Boolean accessor: numeric -> boolean coverage + // ------------------------------------------------------------------ + + /** + * Verifies that {@code getBoolean} converts numeric ClickHouse columns to + * boolean using the "non-zero is true" rule, mirroring + * {@code AbstractBinaryFormatReader} semantics. The test pushes literal + * 0/1 values through several integer widths and also exercises the + * native Bool column. + */ + @Test(groups = {"integration"}) + public void testGetBooleanFromNumericValues() throws Exception { + String sql = "SELECT " + + "toInt8(0) AS v_int8_zero, toInt8(1) AS v_int8_one, " + + "toInt32(0) AS v_int32_zero, toInt32(42) AS v_int32_nonzero, " + + "toInt64(0) AS v_int64_zero, toInt64(-7) AS v_int64_nonzero, " + + "toUInt64(1) AS v_uint64_one, " + + "true AS v_bool_true, false AS v_bool_false"; + + try (QueryResponse response = + client.query(sql, newJsonEachRowSettingsForPrimitives()).get(); + ClickHouseTextFormatReader reader = createReader(response)) { + + Assert.assertNotNull(reader.next()); + + Assert.assertFalse(reader.getBoolean("v_int8_zero")); + Assert.assertTrue(reader.getBoolean("v_int8_one")); + Assert.assertFalse(reader.getBoolean("v_int32_zero")); + Assert.assertTrue(reader.getBoolean("v_int32_nonzero")); + Assert.assertFalse(reader.getBoolean("v_int64_zero")); + Assert.assertTrue(reader.getBoolean("v_int64_nonzero")); + Assert.assertTrue(reader.getBoolean("v_uint64_one")); + Assert.assertTrue(reader.getBoolean("v_bool_true")); + Assert.assertFalse(reader.getBoolean("v_bool_false")); + + // The same values must also be readable by 1-based column index. + Assert.assertFalse(reader.getBoolean(1)); + Assert.assertTrue(reader.getBoolean(2)); + } + } + + // ------------------------------------------------------------------ + // Array accessor coverage + // ------------------------------------------------------------------ + + /** + * Exercises the typed Array accessors against a row of mixed Array(...) + * columns. Verifies that: + * + *
    + *
  • {@code getList} returns the parser-native list.
  • + *
  • The typed primitive accessors coerce parsed JSON numbers into the + * requested primitive type, regardless of whether the parser + * materialized elements as {@code Integer}, {@code Long}, + * {@code Double}, or {@code BigDecimal}.
  • + *
  • The typed accessors throw on non-array columns.
  • + *
+ */ + @Test(groups = {"integration"}) + public void testArrayAccessors() throws Exception { + String sql = "SELECT " + + "[1, 2, 3]::Array(Int32) AS col_int_arr, " + + "[10, 20, 30]::Array(Int64) AS col_long_arr, " + + "[1, 2]::Array(Int16) AS col_short_arr, " + + "[7, 8]::Array(Int8) AS col_byte_arr, " + + "[1.5, 2.5]::Array(Float64) AS col_double_arr, " + + "[1.0, 2.0]::Array(Float32) AS col_float_arr, " + + "['a', 'b', 'c']::Array(String) AS col_string_arr, " + + "[true, false, true]::Array(Bool) AS col_bool_arr, " + + "toInt32(1) AS col_not_array"; + + try (QueryResponse response = + client.query(sql, newJsonEachRowSettingsForPrimitives()).get(); + ClickHouseTextFormatReader reader = createReader(response)) { + + Assert.assertNotNull(reader.next()); + + List intList = reader.getList("col_int_arr"); + Assert.assertNotNull(intList); + Assert.assertEquals(intList.size(), 3); + + // Each typed accessor is exercised by name first and then by 1-based + // column index so the index-based overloads are also covered against + // the inferred schema produced from the first row. + Assert.assertEquals(reader.getIntArray("col_int_arr"), new int[] {1, 2, 3}); + Assert.assertEquals(reader.getIntArray(1), new int[] {1, 2, 3}); + Assert.assertEquals(reader.getLongArray("col_long_arr"), new long[] {10L, 20L, 30L}); + Assert.assertEquals(reader.getLongArray(2), new long[] {10L, 20L, 30L}); + Assert.assertEquals(reader.getShortArray("col_short_arr"), + new short[] {(short) 1, (short) 2}); + Assert.assertEquals(reader.getShortArray(3), new short[] {(short) 1, (short) 2}); + Assert.assertEquals(reader.getByteArray("col_byte_arr"), new byte[] {(byte) 7, (byte) 8}); + Assert.assertEquals(reader.getByteArray(4), new byte[] {(byte) 7, (byte) 8}); + Assert.assertEquals(reader.getDoubleArray("col_double_arr"), + new double[] {1.5d, 2.5d}, 1e-9); + Assert.assertEquals(reader.getDoubleArray(5), new double[] {1.5d, 2.5d}, 1e-9); + Assert.assertEquals(reader.getFloatArray("col_float_arr"), + new float[] {1.0f, 2.0f}, 1e-6f); + Assert.assertEquals(reader.getFloatArray(6), new float[] {1.0f, 2.0f}, 1e-6f); + Assert.assertEquals(reader.getStringArray("col_string_arr"), + new String[] {"a", "b", "c"}); + Assert.assertEquals(reader.getStringArray(7), new String[] {"a", "b", "c"}); + Assert.assertEquals(reader.getBooleanArray("col_bool_arr"), + new boolean[] {true, false, true}); + Assert.assertEquals(reader.getBooleanArray(8), new boolean[] {true, false, true}); + + Assert.assertEquals(reader.getObjectArray("col_int_arr").length, 3); + Assert.assertEquals(reader.getObjectArray(1).length, 3); + + // Non-array columns must surface a RuntimeException rather than + // silently returning null or a malformed array. + try { + reader.getIntArray("col_not_array"); + Assert.fail("Expected exception on scalar column"); + } catch (RuntimeException expected) { + // ok + } + try { + reader.getList("col_not_array"); + Assert.fail("Expected exception on scalar column"); + } catch (RuntimeException expected) { + // ok + } + } + } + + /** + * Locks in the contract that a {@code null} element coming from the server + * (e.g. through {@code Array(Nullable(...))}) cannot be silently turned + * into a zero/false slot of a Java primitive array. The reader must throw + * a {@link RuntimeException} so callers don't lose the distinction between + * "missing value" and "actual zero". + */ + @Test(groups = {"integration"}) + public void testArrayAccessorsRejectNullElementsFromServer() throws Exception { + String sql = "SELECT " + + "[1, NULL, 3]::Array(Nullable(Int32)) AS col_int_arr, " + + "[true, NULL]::Array(Nullable(Bool)) AS col_bool_arr, " + + "['x', NULL, 'z']::Array(Nullable(String)) AS col_string_arr"; + + try (QueryResponse response = + client.query(sql, newJsonEachRowSettingsForPrimitives()).get(); + ClickHouseTextFormatReader reader = createReader(response)) { + + Assert.assertNotNull(reader.next()); + + try { + reader.getIntArray("col_int_arr"); + Assert.fail("Expected exception on null element in primitive array"); + } catch (RuntimeException expected) { + // ok + } + try { + reader.getBooleanArray("col_bool_arr"); + Assert.fail("Expected exception on null element in primitive array"); + } catch (RuntimeException expected) { + // ok + } + + // getStringArray and getObjectArray preserve nulls because the + // resulting array can hold null references. + Assert.assertEquals(reader.getStringArray("col_string_arr"), + new String[] {"x", null, "z"}); + Object[] objs = reader.getObjectArray("col_string_arr"); + Assert.assertEquals(objs.length, 3); + Assert.assertNull(objs[1]); + + // getList must surface the raw list with the null element intact. + List intList = reader.getList("col_int_arr"); + Assert.assertNotNull(intList); + Assert.assertEquals(intList.size(), 3); + Assert.assertNull(intList.get(1)); + } + } + + // ------------------------------------------------------------------ + // Test case definitions for primitive types + // ------------------------------------------------------------------ + + /** + * Describes one primitive ClickHouse column under test. Each case carries the SQL + * literals used to populate the row, the expected Java values, the accessor + * functions that are expected to succeed, and a list of accessors that must throw + * for values of this type. + */ + public static final class PrimitiveTypeCase { + final String columnName; + final String chType; + final List sqlLiterals; + final List expectedValues; + final BiFunction readByName; + final BiFunction readByIndex; + final BiPredicate equality; + final List incompatibleAccessors; + + PrimitiveTypeCase(String columnName, String chType, + List sqlLiterals, + List expectedValues, + BiFunction readByName, + BiFunction readByIndex, + BiPredicate equality, + List incompatibleAccessors) { + this.columnName = columnName; + this.chType = chType; + this.sqlLiterals = sqlLiterals; + this.expectedValues = expectedValues; + this.readByName = readByName; + this.readByIndex = readByIndex; + this.equality = equality; + this.incompatibleAccessors = incompatibleAccessors; + } + + @Override + public String toString() { + return columnName + " (" + chType + ")"; + } + } + + /** Pair of accessor invocations (by name and by index) that must throw for the case's column. */ + public static final class IncompatibleAccessor { + final String name; + final Consumer byName; + final Consumer byIndex; + + IncompatibleAccessor(String name, + Consumer byName, + Consumer byIndex) { + this.name = name; + this.byName = byName; + this.byIndex = byIndex; + } + } + + private static List buildPrimitiveCases(Random rnd) { + List cases = new ArrayList<>(); + + // ---- Signed integers --------------------------------------------------- + cases.add(intCase("col_int8", "Int8", + Arrays.asList( + Byte.MIN_VALUE, Byte.MAX_VALUE, (byte) 0, + (byte) (rnd.nextInt(256) - 128), + (byte) (rnd.nextInt(256) - 128)), + Number::byteValue, + (r, n) -> r.getByte(n), + (r, i) -> r.getByte(i))); + + cases.add(intCase("col_int16", "Int16", + Arrays.asList( + Short.MIN_VALUE, Short.MAX_VALUE, (short) 0, + (short) (rnd.nextInt(65536) - 32768), + (short) (rnd.nextInt(65536) - 32768)), + Number::shortValue, + (r, n) -> r.getShort(n), + (r, i) -> r.getShort(i))); + + cases.add(intCase("col_int32", "Int32", + Arrays.asList( + Integer.MIN_VALUE, Integer.MAX_VALUE, 0, + rnd.nextInt(), rnd.nextInt()), + Number::intValue, + (r, n) -> r.getInteger(n), + (r, i) -> r.getInteger(i))); + + cases.add(intCase("col_int64", "Int64", + Arrays.asList( + Long.MIN_VALUE, Long.MAX_VALUE, 0L, + rnd.nextLong(), rnd.nextLong()), + Number::longValue, + (r, n) -> r.getLong(n), + (r, i) -> r.getLong(i))); + + // ---- Unsigned integers ------------------------------------------------- + cases.add(intCase("col_uint8", "UInt8", + Arrays.asList( + (short) 0, (short) 255, (short) 128, + (short) rnd.nextInt(256), + (short) rnd.nextInt(256)), + Number::shortValue, + (r, n) -> r.getShort(n), + (r, i) -> r.getShort(i))); + + cases.add(intCase("col_uint16", "UInt16", + Arrays.asList( + 0, 65535, 32768, + rnd.nextInt(65536), + rnd.nextInt(65536)), + Number::intValue, + (r, n) -> r.getInteger(n), + (r, i) -> r.getInteger(i))); + + cases.add(intCase("col_uint32", "UInt32", + Arrays.asList( + 0L, 4294967295L, 1L, + (long) rnd.nextInt() & 0xFFFFFFFFL, + (long) rnd.nextInt() & 0xFFFFFFFFL), + Number::longValue, + (r, n) -> r.getLong(n), + (r, i) -> r.getLong(i))); + + // UInt64 max (2^64 - 1) does not fit into a signed long, so we read it as + // BigInteger and use BigInteger equality everywhere. + List uint64Values = Arrays.asList( + BigInteger.ZERO, + new BigInteger("18446744073709551615"), + BigInteger.ONE, + new BigInteger(63, rnd), + new BigInteger(64, rnd)); + cases.add(new PrimitiveTypeCase( + "col_uint64", "UInt64", + sqlLiteralsFromValues(uint64Values, v -> "toUInt64('" + v + "')"), + uint64Values, + (r, n) -> r.getBigInteger(n), + (r, i) -> r.getBigInteger(i), + AbstractJSONEachRowFormatReaderTests::equalsByEquals, + incompatibleForNumericValue("col_uint64", 1))); + + // ---- Floating-point ---------------------------------------------------- + List floatValues = Arrays.asList( + 0.0f, Float.MAX_VALUE, -Float.MAX_VALUE, + rnd.nextFloat() * 1_000f - 500f, + rnd.nextFloat() * 1_000f - 500f); + cases.add(new PrimitiveTypeCase( + "col_float32", "Float32", + sqlLiteralsFromValues(floatValues, + v -> "toFloat32(" + Float.toString(v) + ")"), + toObjectList(floatValues), + (r, n) -> r.getFloat(n), + (r, i) -> r.getFloat(i), + AbstractJSONEachRowFormatReaderTests::approximatelyEqualsFloat, + incompatibleForNumericValue("col_float32", 1))); + + List doubleValues = Arrays.asList( + 0.0d, Double.MAX_VALUE, -Double.MAX_VALUE, + rnd.nextDouble() * 1_000d - 500d, + rnd.nextDouble() * 1_000d - 500d); + cases.add(new PrimitiveTypeCase( + "col_float64", "Float64", + sqlLiteralsFromValues(doubleValues, + v -> "toFloat64(" + Double.toString(v) + ")"), + toObjectList(doubleValues), + (r, n) -> r.getDouble(n), + (r, i) -> r.getDouble(i), + AbstractJSONEachRowFormatReaderTests::approximatelyEqualsDouble, + incompatibleForNumericValue("col_float64", 1))); + + // ---- Decimal ----------------------------------------------------------- + List decimalValues = Arrays.asList( + new BigDecimal("0.0000"), + new BigDecimal("99999999999999.9999"), + new BigDecimal("-99999999999999.9999"), + new BigDecimal(rnd.nextLong() % 1_000_000_000L) + .movePointLeft(4), + new BigDecimal(rnd.nextLong() % 1_000_000_000L) + .movePointLeft(4)); + cases.add(new PrimitiveTypeCase( + "col_decimal", "Decimal(18, 4)", + sqlLiteralsFromValues(decimalValues, + v -> "toDecimal64('" + v.toPlainString() + "', 4)"), + toObjectList(decimalValues), + (r, n) -> r.getBigDecimal(n), + (r, i) -> r.getBigDecimal(i), + AbstractJSONEachRowFormatReaderTests::equalsBigDecimal, + incompatibleForNumericValue("col_decimal", 1))); + + // ---- Bool -------------------------------------------------------------- + List boolValues = Arrays.asList(false, true, false, rnd.nextBoolean(), rnd.nextBoolean()); + cases.add(new PrimitiveTypeCase( + "col_bool", "Bool", + sqlLiteralsFromValues(boolValues, Object::toString), + toObjectList(boolValues), + (r, n) -> r.getBoolean(n), + (r, i) -> r.getBoolean(i), + AbstractJSONEachRowFormatReaderTests::equalsByEquals, + incompatibleForBoolValue("col_bool", 1))); + + // ---- String ------------------------------------------------------------ + List stringValues = Arrays.asList( + "", + "hello world", + randomAsciiString(rnd, 32), + randomAsciiString(rnd, 16), + "line1\nline2\twith special chars: 'quoted'"); + cases.add(new PrimitiveTypeCase( + "col_string", "String", + sqlLiteralsFromValues(stringValues, + AbstractJSONEachRowFormatReaderTests::toClickHouseStringLiteral), + toObjectList(stringValues), + (r, n) -> r.getString(n), + (r, i) -> r.getString(i), + AbstractJSONEachRowFormatReaderTests::equalsByEquals, + incompatibleForStringValue("col_string", 1))); + + // ---- Date -------------------------------------------------------------- + List dateValues = Arrays.asList( + LocalDate.of(1970, 1, 1), // Date min + LocalDate.of(2149, 6, 6), // Date max + LocalDate.of(2000, 1, 1), + randomDate(rnd), + randomDate(rnd)); + cases.add(new PrimitiveTypeCase( + "col_date", "Date", + sqlLiteralsFromValues(dateValues, v -> "toDate('" + v + "')"), + toObjectList(dateValues), + (r, n) -> r.getLocalDate(n), + (r, i) -> r.getLocalDate(i), + AbstractJSONEachRowFormatReaderTests::equalsByEquals, + incompatibleForDateValue("col_date", 1))); + + // ---- UUID -------------------------------------------------------------- + List uuidValues = Arrays.asList( + new UUID(0L, 0L), + new UUID(-1L, -1L), + UUID.fromString("11111111-2222-3333-4444-555555555555"), + new UUID(rnd.nextLong(), rnd.nextLong()), + new UUID(rnd.nextLong(), rnd.nextLong())); + cases.add(new PrimitiveTypeCase( + "col_uuid", "UUID", + sqlLiteralsFromValues(uuidValues, v -> "toUUID('" + v + "')"), + toObjectList(uuidValues), + (r, n) -> r.getUUID(n), + (r, i) -> r.getUUID(i), + AbstractJSONEachRowFormatReaderTests::equalsByEquals, + incompatibleForUuidValue("col_uuid", 1))); + + // ---- Enum8 / Enum16 ---------------------------------------------------- + // ClickHouse serialises enum columns into JSONEachRow as their string + // labels (not as the underlying numeric value). The reader therefore + // exposes them through getString, while numeric and temporal accessors + // remain incompatible. The 1-based column index passed to the + // incompatible-accessor helper is always 1 because each parameterized + // run selects a single column. + List enum8Values = Arrays.asList( + "red", "green", "blue", "red", "green"); + cases.add(new PrimitiveTypeCase( + "col_enum8", "Enum8('red' = 1, 'green' = 2, 'blue' = 3)", + sqlLiteralsFromValues(enum8Values, + AbstractJSONEachRowFormatReaderTests::toClickHouseStringLiteral), + toObjectList(enum8Values), + (r, n) -> r.getString(n), + (r, i) -> r.getString(i), + AbstractJSONEachRowFormatReaderTests::equalsByEquals, + incompatibleForEnumValue("col_enum8", 1))); + + List enum16Values = Arrays.asList( + "alpha", "beta", "gamma", "alpha", "beta"); + cases.add(new PrimitiveTypeCase( + "col_enum16", "Enum16('alpha' = 100, 'beta' = 200, 'gamma' = 300)", + sqlLiteralsFromValues(enum16Values, + AbstractJSONEachRowFormatReaderTests::toClickHouseStringLiteral), + toObjectList(enum16Values), + (r, n) -> r.getString(n), + (r, i) -> r.getString(i), + AbstractJSONEachRowFormatReaderTests::equalsByEquals, + incompatibleForEnumValue("col_enum16", 1))); + + return cases; + } + + // ------------------------------------------------------------------ + // Case factories and helpers + // ------------------------------------------------------------------ + + /** + * Builds a case for a signed/unsigned integer column whose expected Java type is a + * boxed integer. Values are passed via {@code toXxx('literal')} casts so the SQL + * parser does not need to evaluate large unary expressions. + */ + private static PrimitiveTypeCase intCase( + String columnName, String chType, + List values, + java.util.function.Function normalize, + BiFunction readByName, + BiFunction readByIndex) { + + List expected = new ArrayList<>(values.size()); + for (Number v : values) { + expected.add(normalize.apply(v)); + } + List literals = sqlLiteralsFromValues(values, + v -> "to" + chType + "('" + v + "')"); + return new PrimitiveTypeCase(columnName, chType, literals, expected, + readByName, readByIndex, + AbstractJSONEachRowFormatReaderTests::equalsByEquals, + incompatibleForNumericValue(columnName, 1)); + } + + private static List sqlLiteralsFromValues(List values, + java.util.function.Function toLiteral) { + List literals = new ArrayList<>(values.size()); + for (T v : values) { + literals.add(toLiteral.apply(v)); + } + return literals; + } + + private static List toObjectList(List values) { + return new ArrayList<>(values); + } + + // ---- Equality helpers ------------------------------------------------------ + + private static boolean equalsByEquals(Object actual, Object expected) { + if (expected == null) { + return actual == null; + } + return expected.equals(actual); + } + + private static boolean equalsBigDecimal(Object actual, Object expected) { + if (expected == null) { + return actual == null; + } + if (!(actual instanceof BigDecimal)) { + return false; + } + return ((BigDecimal) expected).compareTo((BigDecimal) actual) == 0; + } + + private static boolean approximatelyEqualsFloat(Object actual, Object expected) { + if (actual == null || expected == null) { + return actual == expected; + } + float a = ((Number) actual).floatValue(); + float e = ((Number) expected).floatValue(); + if (Float.compare(a, e) == 0) { + return true; + } + if (Float.isInfinite(a) || Float.isInfinite(e) || Float.isNaN(a) || Float.isNaN(e)) { + return Float.compare(a, e) == 0; + } + float tolerance = Math.max(Math.ulp(e) * 4f, Math.abs(e) * 1e-6f); + return Math.abs(a - e) <= tolerance; + } + + private static boolean approximatelyEqualsDouble(Object actual, Object expected) { + if (actual == null || expected == null) { + return actual == expected; + } + double a = ((Number) actual).doubleValue(); + double e = ((Number) expected).doubleValue(); + if (Double.compare(a, e) == 0) { + return true; + } + if (Double.isInfinite(a) || Double.isInfinite(e) || Double.isNaN(a) || Double.isNaN(e)) { + return Double.compare(a, e) == 0; + } + double tolerance = Math.max(Math.ulp(e) * 4d, Math.abs(e) * 1e-12d); + return Math.abs(a - e) <= tolerance; + } + + // ---- Incompatible accessor sets ------------------------------------------- + + /** + * Accessors that always fail (or fail for non-numeric content). For numeric columns + * we rely on date/UUID/temporal accessors to throw because the value cannot be + * parsed as a date or UUID. For unsupported accessors we expect + * {@link UnsupportedOperationException} from the reader implementation. + */ + private static List incompatibleForNumericValue(String name, int index) { + return Arrays.asList( + new IncompatibleAccessor("getLocalDate", + r -> r.getLocalDate(name), + r -> r.getLocalDate(index)), + new IncompatibleAccessor("getLocalTime", + r -> r.getLocalTime(name), + r -> r.getLocalTime(index)), + new IncompatibleAccessor("getLocalDateTime", + r -> r.getLocalDateTime(name), + r -> r.getLocalDateTime(index)), + new IncompatibleAccessor("getUUID", + r -> r.getUUID(name), + r -> r.getUUID(index)), + new IncompatibleAccessor("getZonedDateTime", + r -> r.getZonedDateTime(name), + r -> r.getZonedDateTime(index)), + new IncompatibleAccessor("getInstant", + r -> r.getInstant(name), + r -> r.getInstant(index)), + new IncompatibleAccessor("getInet4Address", + r -> r.getInet4Address(name), + r -> r.getInet4Address(index))); + } + + private static List incompatibleForBoolValue(String name, int index) { + // Numeric accessors fail because Boolean is not a Number, and string parsers + // fail because "true" / "false" cannot be parsed as a number, date, or UUID. + return Arrays.asList( + new IncompatibleAccessor("getByte", + r -> r.getByte(name), + r -> r.getByte(index)), + new IncompatibleAccessor("getInteger", + r -> r.getInteger(name), + r -> r.getInteger(index)), + new IncompatibleAccessor("getLong", + r -> r.getLong(name), + r -> r.getLong(index)), + new IncompatibleAccessor("getBigInteger", + r -> r.getBigInteger(name), + r -> r.getBigInteger(index)), + new IncompatibleAccessor("getBigDecimal", + r -> r.getBigDecimal(name), + r -> r.getBigDecimal(index)), + new IncompatibleAccessor("getLocalDate", + r -> r.getLocalDate(name), + r -> r.getLocalDate(index)), + new IncompatibleAccessor("getUUID", + r -> r.getUUID(name), + r -> r.getUUID(index)), + new IncompatibleAccessor("getZonedDateTime", + r -> r.getZonedDateTime(name), + r -> r.getZonedDateTime(index))); + } + + private static List incompatibleForStringValue(String name, int index) { + // String content here is not numeric, not a date, and not a UUID, so numeric + // and temporal accessors must throw. getBoolean is also expected to throw + // because the JSONEachRow reader does not silently convert string content + // through Boolean.parseBoolean (matching AbstractBinaryFormatReader-style + // strictness about incompatible types). + return Arrays.asList( + new IncompatibleAccessor("getByte", + r -> r.getByte(name), + r -> r.getByte(index)), + new IncompatibleAccessor("getInteger", + r -> r.getInteger(name), + r -> r.getInteger(index)), + new IncompatibleAccessor("getLong", + r -> r.getLong(name), + r -> r.getLong(index)), + new IncompatibleAccessor("getDouble", + r -> r.getDouble(name), + r -> r.getDouble(index)), + new IncompatibleAccessor("getBoolean", + r -> r.getBoolean(name), + r -> r.getBoolean(index)), + new IncompatibleAccessor("getBigInteger", + r -> r.getBigInteger(name), + r -> r.getBigInteger(index)), + new IncompatibleAccessor("getBigDecimal", + r -> r.getBigDecimal(name), + r -> r.getBigDecimal(index)), + new IncompatibleAccessor("getLocalDate", + r -> r.getLocalDate(name), + r -> r.getLocalDate(index)), + new IncompatibleAccessor("getUUID", + r -> r.getUUID(name), + r -> r.getUUID(index)), + new IncompatibleAccessor("getZonedDateTime", + r -> r.getZonedDateTime(name), + r -> r.getZonedDateTime(index))); + } + + private static List incompatibleForDateValue(String name, int index) { + // Date columns arrive as strings, so numeric accessors throw ClassCastException + // and time-only / date-time accessors fail to parse the YYYY-MM-DD shape. + return Arrays.asList( + new IncompatibleAccessor("getInteger", + r -> r.getInteger(name), + r -> r.getInteger(index)), + new IncompatibleAccessor("getLong", + r -> r.getLong(name), + r -> r.getLong(index)), + new IncompatibleAccessor("getDouble", + r -> r.getDouble(name), + r -> r.getDouble(index)), + new IncompatibleAccessor("getBigDecimal", + r -> r.getBigDecimal(name), + r -> r.getBigDecimal(index)), + new IncompatibleAccessor("getLocalTime", + r -> r.getLocalTime(name), + r -> r.getLocalTime(index)), + new IncompatibleAccessor("getLocalDateTime", + r -> r.getLocalDateTime(name), + r -> r.getLocalDateTime(index)), + new IncompatibleAccessor("getUUID", + r -> r.getUUID(name), + r -> r.getUUID(index)), + new IncompatibleAccessor("getZonedDateTime", + r -> r.getZonedDateTime(name), + r -> r.getZonedDateTime(index))); + } + + private static List incompatibleForEnumValue(String name, int index) { + // ClickHouse serialises Enum8/Enum16 columns as their string label in + // JSONEachRow output, so numeric and temporal accessors must throw and + // getBoolean must also throw (the reader does not silently parse + // string content into a boolean). getEnum8/getEnum16 forward to + // getByte/getShort, so they must throw on a string value as well. + return Arrays.asList( + new IncompatibleAccessor("getByte", + r -> r.getByte(name), + r -> r.getByte(index)), + new IncompatibleAccessor("getShort", + r -> r.getShort(name), + r -> r.getShort(index)), + new IncompatibleAccessor("getInteger", + r -> r.getInteger(name), + r -> r.getInteger(index)), + new IncompatibleAccessor("getLong", + r -> r.getLong(name), + r -> r.getLong(index)), + new IncompatibleAccessor("getDouble", + r -> r.getDouble(name), + r -> r.getDouble(index)), + new IncompatibleAccessor("getBoolean", + r -> r.getBoolean(name), + r -> r.getBoolean(index)), + new IncompatibleAccessor("getBigInteger", + r -> r.getBigInteger(name), + r -> r.getBigInteger(index)), + new IncompatibleAccessor("getBigDecimal", + r -> r.getBigDecimal(name), + r -> r.getBigDecimal(index)), + new IncompatibleAccessor("getEnum8", + r -> r.getEnum8(name), + r -> r.getEnum8(index)), + new IncompatibleAccessor("getEnum16", + r -> r.getEnum16(name), + r -> r.getEnum16(index)), + new IncompatibleAccessor("getLocalDate", + r -> r.getLocalDate(name), + r -> r.getLocalDate(index)), + new IncompatibleAccessor("getUUID", + r -> r.getUUID(name), + r -> r.getUUID(index))); + } + + private static List incompatibleForUuidValue(String name, int index) { + return Arrays.asList( + new IncompatibleAccessor("getInteger", + r -> r.getInteger(name), + r -> r.getInteger(index)), + new IncompatibleAccessor("getLong", + r -> r.getLong(name), + r -> r.getLong(index)), + new IncompatibleAccessor("getDouble", + r -> r.getDouble(name), + r -> r.getDouble(index)), + new IncompatibleAccessor("getBigDecimal", + r -> r.getBigDecimal(name), + r -> r.getBigDecimal(index)), + new IncompatibleAccessor("getLocalDate", + r -> r.getLocalDate(name), + r -> r.getLocalDate(index)), + new IncompatibleAccessor("getLocalDateTime", + r -> r.getLocalDateTime(name), + r -> r.getLocalDateTime(index)), + new IncompatibleAccessor("getZonedDateTime", + r -> r.getZonedDateTime(name), + r -> r.getZonedDateTime(index))); + } + + // ---- Random value helpers ------------------------------------------------- + + private static String randomAsciiString(Random rnd, int length) { + StringBuilder sb = new StringBuilder(length); + for (int i = 0; i < length; i++) { + // Printable ASCII range except quote (32-126 except 39 and 92). + char c; + do { + c = (char) (32 + rnd.nextInt(95)); + } while (c == '\'' || c == '\\'); + sb.append(c); + } + return sb.toString(); + } + + private static LocalDate randomDate(Random rnd) { + // Stay well inside the Date range (1970-01-01..2149-06-06) to avoid time-zone + // edge effects when the server re-serialises the value into JSON. + int year = 1971 + rnd.nextInt(170); + int month = 1 + rnd.nextInt(12); + int day = 1 + rnd.nextInt(LocalDate.of(year, month, 1).lengthOfMonth()); + return LocalDate.of(year, month, day); + } + + private static String toClickHouseStringLiteral(String value) { + StringBuilder sb = new StringBuilder(value.length() + 2); + sb.append('\''); + for (int i = 0; i < value.length(); i++) { + char c = value.charAt(i); + switch (c) { + case '\'': + sb.append("\\'"); + break; + case '\\': + sb.append("\\\\"); + break; + case '\n': + sb.append("\\n"); + break; + case '\t': + sb.append("\\t"); + break; + case '\r': + sb.append("\\r"); + break; + default: + sb.append(c); + break; + } + } + sb.append('\''); + return sb.toString(); + } +} diff --git a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/GsonJSONEachRowFormatReaderTests.java b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/GsonJSONEachRowFormatReaderTests.java new file mode 100644 index 000000000..57f4e1abe --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/GsonJSONEachRowFormatReaderTests.java @@ -0,0 +1,74 @@ +package com.clickhouse.client.api.data_formats; + +import com.clickhouse.client.api.insert.InsertResponse; +import com.clickhouse.client.api.insert.InsertSettings; +import com.clickhouse.client.api.query.QueryResponse; +import com.clickhouse.data.ClickHouseFormat; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.stream.JsonWriter; +import lombok.AllArgsConstructor; +import lombok.Data; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStreamWriter; +import java.util.Collections; +import java.util.Map; + +@Test(groups = {"integration"}) +public class GsonJSONEachRowFormatReaderTests extends AbstractJSONEachRowFormatReaderTests { + + private static Gson gson = new GsonBuilder().create(); + + private JsonParserFactory parserFactory = new GsonJsonParserFactory(); + + @Override + protected ClickHouseTextFormatReader createReader(QueryResponse response) throws IOException { + return new JSONEachRowFormatReader(parserFactory.createJsonParser(response.getInputStream())); + } + + @Override + protected ClickHouseTextFormatReader createReader(InputStream input) throws IOException { + return new JSONEachRowFormatReader(parserFactory.createJsonParser(input)); + } + + @Test(groups = {"integration"}) + public void testRowToObject() throws Exception { + + TestDTO_1[] data = new TestDTO_1[] { + new TestDTO_1("key1", 0.2, -0.2, Collections.singletonMap("p1", 10)), + new TestDTO_1("key2", 0.3, -0.5, Collections.singletonMap("p1", 9)), + }; + + final String table = "test_row_to_object_json"; + final String createStmt = "CREATE TABLE IF NOT EXISTS " + table + " (key String, sensor1 Decimal, sensor2 Decimal, params JSON) Engine MergeTree Order By (key)"; + client.execute(createStmt).get().close(); + client.execute("TRUNCATE " + table).get().close(); + + try (InsertResponse response = client.insert(table, out -> { + try (JsonWriter jsonWriter = gson.newJsonWriter(new OutputStreamWriter(out))) { + jsonWriter.setLenient(true); + for (TestDTO_1 value : data) { + gson.toJson(value, TestDTO_1.class, jsonWriter); + } + } + }, ClickHouseFormat.JSONEachRow, new InsertSettings()).get()) { + System.out.println("inserted rows" + response.getWrittenRows() ); + } + } + + @Data + @AllArgsConstructor + public static class TestDTO_1 { + + private String key; + + private Double sensor1; + + private Double sensor2; + + private Map params; + } +} diff --git a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/JSONEachRowFormatReaderTest.java b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/JSONEachRowFormatReaderTest.java new file mode 100644 index 000000000..4e0220cc7 --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/JSONEachRowFormatReaderTest.java @@ -0,0 +1,795 @@ +package com.clickhouse.client.api.data_formats; + +import com.clickhouse.data.ClickHouseDataType; +import org.testng.Assert; +import org.testng.annotations.Test; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; + +@Test(groups = {"unit"}) +public class JSONEachRowFormatReaderTest { + + /** Simple in-memory parser that yields a fixed list of rows. */ + private static final class StubJsonParser implements JsonParser { + private final List> rows; + private boolean closed; + private int index; + + StubJsonParser(List> rows) { + this.rows = new ArrayList<>(rows); + } + + @Override + public Map nextRow() { + return index < rows.size() ? rows.get(index++) : null; + } + + @Override + public void close() { + closed = true; + } + + boolean isClosed() { + return closed; + } + } + + private static Map row(Object... pairs) { + Map r = new LinkedHashMap<>(); + for (int i = 0; i < pairs.length; i += 2) { + r.put((String) pairs[i], pairs[i + 1]); + } + return r; + } + + private static JSONEachRowFormatReader readerOf(Map... rows) { + return new JSONEachRowFormatReader(new StubJsonParser(Arrays.asList(rows))); + } + + // --------------------------------------------------------------------- + // Schema inference + // --------------------------------------------------------------------- + + @Test + public void testSchemaInferenceForIntegerLikeValues() { + JSONEachRowFormatReader reader = readerOf(row( + "as_integer", 1, + "as_long", 2L, + "as_big_integer", BigInteger.TEN)); + + Assert.assertEquals(reader.getSchema().getColumnByName("as_integer").getDataType(), + ClickHouseDataType.Int32); + Assert.assertEquals(reader.getSchema().getColumnByName("as_long").getDataType(), + ClickHouseDataType.Int64); + Assert.assertEquals(reader.getSchema().getColumnByName("as_big_integer").getDataType(), + ClickHouseDataType.Int256); + } + + @Test + public void testSchemaInferenceForFractionalValues() { + JSONEachRowFormatReader reader = readerOf(row( + "as_double", 1.5d, + "as_float", 2.5f, + "as_big_decimal", new BigDecimal("3.14"))); + + Assert.assertEquals(reader.getSchema().getColumnByName("as_double").getDataType(), + ClickHouseDataType.Float64); + Assert.assertEquals(reader.getSchema().getColumnByName("as_float").getDataType(), + ClickHouseDataType.Float32); + Assert.assertEquals(reader.getSchema().getColumnByName("as_big_decimal").getDataType(), + ClickHouseDataType.Decimal); + } + + @Test + public void testSchemaInferenceUsesJavaTypeForWholeFractionalValues() { + JSONEachRowFormatReader reader = readerOf(row( + "as_double_whole", 5.0d, + "as_float_whole", 7.0f, + "as_big_decimal_whole", new BigDecimal("42"))); + + Assert.assertEquals(reader.getSchema().getColumnByName("as_double_whole").getDataType(), + ClickHouseDataType.Float64); + Assert.assertEquals(reader.getSchema().getColumnByName("as_float_whole").getDataType(), + ClickHouseDataType.Float32); + Assert.assertEquals(reader.getSchema().getColumnByName("as_big_decimal_whole").getDataType(), + ClickHouseDataType.Decimal); + } + + @Test + public void testGuessDataTypeForOutOfRangeDoubleIsFloat64() { + // Values outside the long range cannot be represented as Int64; the + // reader must fall back to Float64 even when they are mathematically + // whole numbers. + JSONEachRowFormatReader reader = readerOf(row( + "too_big", 1.0e20d, + "infinite", Double.POSITIVE_INFINITY)); + + Assert.assertEquals(reader.getSchema().getColumnByName("too_big").getDataType(), + ClickHouseDataType.Float64); + Assert.assertEquals(reader.getSchema().getColumnByName("infinite").getDataType(), + ClickHouseDataType.Float64); + } + + @Test + public void testSchemaInferenceForUnsupportedNumberSubtypesUsesDefault() { + // AtomicInteger is a Number, but it is not part of ClickHouseDataType.DATA_TYPE_TO_CLASS. + JSONEachRowFormatReader reader = readerOf(row("custom", new AtomicInteger(5))); + Assert.assertEquals(reader.getSchema().getColumnByName("custom").getDataType(), + ClickHouseDataType.String); + } + + @Test + public void testSchemaInferenceForBooleanIsBool() { + JSONEachRowFormatReader reader = readerOf(row("flag", Boolean.TRUE)); + Assert.assertEquals(reader.getSchema().getColumnByName("flag").getDataType(), + ClickHouseDataType.Bool); + } + + @Test + public void testSchemaInferenceForStructuredAndSpecialValues() { + UUID uuid = UUID.fromString("11111111-2222-3333-4444-555555555555"); + JSONEachRowFormatReader reader = readerOf(row( + "as_string", "hello", + "as_list", Arrays.asList(1, 2, 3), + "as_array", new double[] {1.0d, 2.0d}, + "as_map", Collections.singletonMap("k", "v"), + "as_uuid", uuid, + "as_date", LocalDate.of(2024, 1, 2), + "as_datetime", LocalDateTime.of(2024, 1, 2, 3, 4), + "as_null", null)); + + Assert.assertEquals(reader.getSchema().getColumnByName("as_string").getDataType(), + ClickHouseDataType.String); + Assert.assertEquals(reader.getSchema().getColumnByName("as_list").getDataType(), + ClickHouseDataType.Array); + Assert.assertEquals(reader.getSchema().getColumnByName("as_array").getDataType(), + ClickHouseDataType.Array); + Assert.assertEquals(reader.getSchema().getColumnByName("as_map").getDataType(), + ClickHouseDataType.Map); + Assert.assertEquals(reader.getSchema().getColumnByName("as_uuid").getDataType(), + ClickHouseDataType.UUID); + Assert.assertEquals(reader.getSchema().getColumnByName("as_date").getDataType(), + ClickHouseDataType.Date); + Assert.assertEquals(reader.getSchema().getColumnByName("as_datetime").getDataType(), + ClickHouseDataType.DateTime64); + Assert.assertEquals(reader.getSchema().getColumnByName("as_null").getDataType(), + ClickHouseDataType.String); + } + + @Test + public void testEmptyResultSetExposesEmptySchema() { + JSONEachRowFormatReader reader = new JSONEachRowFormatReader( + new StubJsonParser(Collections.emptyList())); + + Assert.assertNotNull(reader.getSchema()); + Assert.assertEquals(reader.getSchema().getColumns().size(), 0); + Assert.assertFalse(reader.hasNext()); + Assert.assertNull(reader.next()); + } + + @Test + public void testReaderInitializationWrapsParserFailures() { + JsonParser failing = new JsonParser() { + @Override + public Map nextRow() throws Exception { + throw new IllegalStateException("boom"); + } + @Override + public void close() { } + }; + try { + new JSONEachRowFormatReader(failing); + Assert.fail("Expected RuntimeException"); + } catch (RuntimeException e) { + Assert.assertTrue(e.getMessage().contains("Failed to initialize JSON reader"), + "Unexpected message: " + e.getMessage()); + Assert.assertTrue(e.getCause() instanceof IllegalStateException); + } + } + + @Test + public void testNextRowFailureIsWrapped() throws Exception { + JsonParser parser = new JsonParser() { + private int call; + @Override + public Map nextRow() { + if (call++ == 0) { + return row("id", 1); + } + throw new IllegalStateException("kaboom"); + } + @Override + public void close() { } + }; + try (JSONEachRowFormatReader reader = new JSONEachRowFormatReader(parser)) { + try { + reader.next(); + Assert.fail("Expected RuntimeException"); + } catch (RuntimeException e) { + Assert.assertTrue(e.getMessage().contains("Failed to read next JSON row"), + "Unexpected message: " + e.getMessage()); + } + } + } + + // --------------------------------------------------------------------- + // Row navigation, readValue, hasValue + // --------------------------------------------------------------------- + + @Test + public void testHasNextAndNext() throws Exception { + Map r1 = row("id", 1); + Map r2 = row("id", 2); + + try (JSONEachRowFormatReader reader = readerOf(r1, r2)) { + Assert.assertTrue(reader.hasNext()); + Assert.assertSame(reader.next(), r1); + Assert.assertTrue(reader.hasNext()); + Assert.assertSame(reader.next(), r2); + Assert.assertFalse(reader.hasNext()); + Assert.assertNull(reader.next()); + } + } + + @Test + public void testReadValueByIndexAndName() throws Exception { + try (JSONEachRowFormatReader reader = readerOf(row("id", 42, "name", "abc"))) { + reader.next(); + Number byIndex = reader.readValue(1); + Assert.assertNotNull(byIndex); + Assert.assertEquals(byIndex.intValue(), 42); + Assert.assertEquals((String) reader.readValue("name"), "abc"); + Assert.assertEquals((String) reader.readValue(2), "abc"); + } + } + + @Test + public void testHasValue() throws Exception { + Map r = new HashMap<>(); + r.put("present", "value"); + r.put("nullable", null); + + try (JSONEachRowFormatReader reader = new JSONEachRowFormatReader( + new StubJsonParser(Collections.singletonList(r)))) { + reader.next(); + Assert.assertTrue(reader.hasValue("present")); + Assert.assertFalse(reader.hasValue("nullable")); + Assert.assertFalse(reader.hasValue("missing")); + // The schema only contains the keys observed in the first row, so + // any column index resolved to a name that is present must be + // truthy and any nullable column must be falsy. + Assert.assertEquals(reader.hasValue(1), reader.hasValue(reader.getSchema().columnIndexToName(1))); + } + } + + @Test + public void testCloseDelegatesToParser() throws Exception { + StubJsonParser parser = new StubJsonParser(Collections.singletonList(row("id", 1))); + JSONEachRowFormatReader reader = new JSONEachRowFormatReader(parser); + Assert.assertFalse(parser.isClosed()); + reader.close(); + Assert.assertTrue(parser.isClosed()); + } + + // --------------------------------------------------------------------- + // Typed accessors + // --------------------------------------------------------------------- + + @Test + public void testNumericAccessors() throws Exception { + try (JSONEachRowFormatReader reader = readerOf(row( + "b", 120, + "s", 30000, + "i", 1_000_000, + "l", 10_000_000_000L, + "f", 1.5d, + "d", 2.5d))) { + reader.next(); + + Assert.assertEquals(reader.getByte("b"), (byte) 120); + Assert.assertEquals(reader.getByte(1), (byte) 120); + Assert.assertEquals(reader.getShort("s"), (short) 30000); + Assert.assertEquals(reader.getShort(2), (short) 30000); + Assert.assertEquals(reader.getInteger("i"), 1_000_000); + Assert.assertEquals(reader.getInteger(3), 1_000_000); + Assert.assertEquals(reader.getLong("l"), 10_000_000_000L); + Assert.assertEquals(reader.getLong(4), 10_000_000_000L); + Assert.assertEquals(reader.getFloat("f"), 1.5f, 0.0001f); + Assert.assertEquals(reader.getFloat(5), 1.5f, 0.0001f); + Assert.assertEquals(reader.getDouble("d"), 2.5d, 0.0001d); + Assert.assertEquals(reader.getDouble(6), 2.5d, 0.0001d); + + Assert.assertEquals(reader.getEnum8("b"), (byte) 120); + Assert.assertEquals(reader.getEnum8(1), (byte) 120); + Assert.assertEquals(reader.getEnum16("s"), (short) 30000); + Assert.assertEquals(reader.getEnum16(2), (short) 30000); + } + } + + @Test + public void testStringAccessor() throws Exception { + Map r = new LinkedHashMap<>(); + r.put("s", "hello"); + r.put("missing", null); + + try (JSONEachRowFormatReader reader = new JSONEachRowFormatReader( + new StubJsonParser(Collections.singletonList(r)))) { + reader.next(); + Assert.assertEquals(reader.getString("s"), "hello"); + Assert.assertEquals(reader.getString(1), "hello"); + Assert.assertNull(reader.getString("missing")); + Assert.assertNull(reader.getString(2)); + } + } + + @Test + public void testBooleanAccessor() throws Exception { + try (JSONEachRowFormatReader reader = readerOf(row( + "from_bool", Boolean.TRUE, + "from_zero", 0, + "from_nonzero", 1, + "from_long", 42L, + "from_big_int", new BigInteger("9000000000"), + "from_big_dec_zero", BigDecimal.ZERO))) { + reader.next(); + Assert.assertTrue(reader.getBoolean("from_bool")); + Assert.assertFalse(reader.getBoolean("from_zero")); + Assert.assertTrue(reader.getBoolean("from_nonzero")); + Assert.assertTrue(reader.getBoolean("from_long")); + Assert.assertTrue(reader.getBoolean("from_big_int")); + Assert.assertFalse(reader.getBoolean("from_big_dec_zero")); + Assert.assertTrue(reader.getBoolean(1)); + } + } + + @Test + public void testBooleanAccessorRejectsStringValue() throws Exception { + // Aligning with AbstractBinaryFormatReader expectations: text values + // that are not numeric or boolean are rejected rather than silently + // funneled through Boolean.parseBoolean, which would silently treat + // any non-"true" string as false. + try (JSONEachRowFormatReader reader = readerOf(row("v", "true"))) { + reader.next(); + try { + reader.getBoolean("v"); + Assert.fail("Expected exception for string value"); + } catch (RuntimeException expected) { + // ok + } + try { + reader.getBoolean(1); + Assert.fail("Expected exception for string value"); + } catch (RuntimeException expected) { + // ok + } + } + } + + @Test + public void testBooleanAccessorRejectsNullValue() throws Exception { + Map r = new LinkedHashMap<>(); + r.put("v", null); + try (JSONEachRowFormatReader reader = new JSONEachRowFormatReader( + new StubJsonParser(Collections.singletonList(r)))) { + reader.next(); + try { + reader.getBoolean("v"); + Assert.fail("Expected exception for null value"); + } catch (RuntimeException expected) { + // ok + } + } + } + + @Test + public void testBigNumberAccessors() throws Exception { + try (JSONEachRowFormatReader reader = readerOf(row( + "from_big_integer", new BigInteger("123456789012345"), + "from_string_int", "987654321", + "from_big_decimal", new BigDecimal("12345.6789"), + "from_string_dec", "0.5"))) { + reader.next(); + + Assert.assertEquals(reader.getBigInteger("from_big_integer"), + new BigInteger("123456789012345")); + Assert.assertEquals(reader.getBigInteger("from_string_int"), + new BigInteger("987654321")); + Assert.assertEquals(reader.getBigInteger(1), new BigInteger("123456789012345")); + Assert.assertNull(reader.getBigInteger("not_a_column")); + + Assert.assertEquals(reader.getBigDecimal("from_big_decimal").compareTo(new BigDecimal("12345.6789")), 0); + Assert.assertEquals(reader.getBigDecimal("from_string_dec").compareTo(new BigDecimal("0.5")), 0); + Assert.assertEquals(reader.getBigDecimal(3).compareTo(new BigDecimal("12345.6789")), 0); + } + } + + @Test + public void testTemporalAccessors() throws Exception { + try (JSONEachRowFormatReader reader = readerOf(row( + "d", "2024-05-06", + "t", "07:08:09", + "dt", "2024-05-06T07:08:09", + "odt", "2024-05-06T07:08:09+02:00"))) { + reader.next(); + + Assert.assertEquals(reader.getLocalDate("d"), java.time.LocalDate.of(2024, 5, 6)); + Assert.assertEquals(reader.getLocalDate(1), java.time.LocalDate.of(2024, 5, 6)); + Assert.assertEquals(reader.getLocalTime("t"), java.time.LocalTime.of(7, 8, 9)); + Assert.assertEquals(reader.getLocalTime(2), java.time.LocalTime.of(7, 8, 9)); + Assert.assertEquals(reader.getLocalDateTime("dt"), + java.time.LocalDateTime.of(2024, 5, 6, 7, 8, 9)); + Assert.assertEquals(reader.getLocalDateTime(3), + java.time.LocalDateTime.of(2024, 5, 6, 7, 8, 9)); + Assert.assertEquals(reader.getOffsetDateTime("odt"), + java.time.OffsetDateTime.parse("2024-05-06T07:08:09+02:00")); + Assert.assertEquals(reader.getOffsetDateTime(4), + java.time.OffsetDateTime.parse("2024-05-06T07:08:09+02:00")); + } + } + + @Test + public void testUuidAndListAccessors() throws Exception { + UUID uuid = UUID.fromString("11111111-2222-3333-4444-555555555555"); + try (JSONEachRowFormatReader reader = readerOf(row( + "u", uuid.toString(), + "arr", Arrays.asList(1, 2, 3), + "tuple", Arrays.asList("a", 1)))) { + reader.next(); + + Assert.assertEquals(reader.getUUID("u"), uuid); + Assert.assertEquals(reader.getUUID(1), uuid); + + List list = reader.getList("arr"); + Assert.assertEquals(list, Arrays.asList(1, 2, 3)); + Assert.assertEquals(reader.getList(2), Arrays.asList(1, 2, 3)); + + Assert.assertEquals(reader.getTuple("tuple"), new Object[] {"a", 1}); + Assert.assertEquals(reader.getTuple(3), new Object[] {"a", 1}); + } + } + + @Test + public void testGetTupleHandlesAllValueShapes() throws Exception { + // Covers every branch of getTuple in one row: + // * "as_null" -> null value: must propagate null instead of NPE + // or fabricating an empty array. + // * "as_list" -> List value: must be copied to a new Object[] + // (also exercised by testUuidAndListAccessors, kept here so the + // three branches are visible side-by-side). + // * "as_array" -> Object[] value: must be returned as-is via the + // (Object[]) cast rather than re-wrapped. + // * "as_scalar" -> incompatible scalar: the unchecked cast must + // surface a ClassCastException so callers cannot silently + // misinterpret a non-tuple value. + Object[] tupleArr = new Object[] {"a", 1, Boolean.TRUE}; + Map r = new LinkedHashMap<>(); + r.put("as_null", null); + r.put("as_list", Arrays.asList("x", 2)); + r.put("as_array", tupleArr); + r.put("as_scalar", "not-a-tuple"); + + try (JSONEachRowFormatReader reader = new JSONEachRowFormatReader( + new StubJsonParser(Collections.singletonList(r)))) { + reader.next(); + + Assert.assertNull(reader.getTuple("as_null")); + Assert.assertNull(reader.getTuple(1)); + + Assert.assertEquals(reader.getTuple("as_list"), new Object[] {"x", 2}); + Assert.assertEquals(reader.getTuple(2), new Object[] {"x", 2}); + + Object[] byName = reader.getTuple("as_array"); + Assert.assertSame(byName, tupleArr, + "Object[] values must be returned as-is, not re-wrapped"); + Assert.assertSame(reader.getTuple(3), tupleArr, + "Object[] values must be returned as-is via the index accessor too"); + + for (Runnable call : new Runnable[] { + () -> reader.getTuple("as_scalar"), + () -> reader.getTuple(4)}) { + try { + call.run(); + Assert.fail("Expected ClassCastException for non-array scalar value"); + } catch (ClassCastException expected) { + // ok + } + } + } + } + + // --------------------------------------------------------------------- + // Unsupported operations + // --------------------------------------------------------------------- + + @Test + public void testUnsupportedAccessorsThrow() throws Exception { + try (JSONEachRowFormatReader reader = readerOf(row("v", "x"))) { + reader.next(); + + assertUnsupported(() -> reader.getInstant("v")); + assertUnsupported(() -> reader.getInstant(1)); + assertUnsupported(() -> reader.getZonedDateTime("v")); + assertUnsupported(() -> reader.getZonedDateTime(1)); + assertUnsupported(() -> reader.getDuration("v")); + assertUnsupported(() -> reader.getDuration(1)); + assertUnsupported(() -> reader.getInet4Address("v")); + assertUnsupported(() -> reader.getInet4Address(1)); + assertUnsupported(() -> reader.getInet6Address("v")); + assertUnsupported(() -> reader.getInet6Address(1)); + assertUnsupported(() -> reader.getGeoPoint("v")); + assertUnsupported(() -> reader.getGeoPoint(1)); + assertUnsupported(() -> reader.getGeoRing("v")); + assertUnsupported(() -> reader.getGeoRing(1)); + assertUnsupported(() -> reader.getGeoPolygon("v")); + assertUnsupported(() -> reader.getGeoPolygon(1)); + assertUnsupported(() -> reader.getGeoMultiPolygon("v")); + assertUnsupported(() -> reader.getGeoMultiPolygon(1)); + assertUnsupported(() -> reader.getClickHouseBitmap("v")); + assertUnsupported(() -> reader.getClickHouseBitmap(1)); + assertUnsupported(() -> reader.getTemporalAmount("v")); + assertUnsupported(() -> reader.getTemporalAmount(1)); + } + } + + @Test + public void testArrayAccessorsReturnNullForNullValue() throws Exception { + Map r = new LinkedHashMap<>(); + r.put("v", null); + try (JSONEachRowFormatReader reader = new JSONEachRowFormatReader( + new StubJsonParser(Collections.singletonList(r)))) { + reader.next(); + + // By name: every array accessor must propagate the null cleanly + // rather than throw NullPointerException or fabricate an empty array. + Assert.assertNull(reader.getList("v")); + Assert.assertNull(reader.getByteArray("v")); + Assert.assertNull(reader.getShortArray("v")); + Assert.assertNull(reader.getIntArray("v")); + Assert.assertNull(reader.getLongArray("v")); + Assert.assertNull(reader.getFloatArray("v")); + Assert.assertNull(reader.getDoubleArray("v")); + Assert.assertNull(reader.getBooleanArray("v")); + Assert.assertNull(reader.getStringArray("v")); + Assert.assertNull(reader.getObjectArray("v")); + + // By 1-based index: the same null-propagation guarantee must hold + // for the index-based overloads, which delegate to the name-based + // implementations through the inferred schema. + Assert.assertNull(reader.getList(1)); + Assert.assertNull(reader.getByteArray(1)); + Assert.assertNull(reader.getShortArray(1)); + Assert.assertNull(reader.getIntArray(1)); + Assert.assertNull(reader.getLongArray(1)); + Assert.assertNull(reader.getFloatArray(1)); + Assert.assertNull(reader.getDoubleArray(1)); + Assert.assertNull(reader.getBooleanArray(1)); + Assert.assertNull(reader.getStringArray(1)); + Assert.assertNull(reader.getObjectArray(1)); + } + } + + @Test + public void testArrayAccessorsRejectNullElement() throws Exception { + // A null element in a JSON array cannot be stored into a Java + // primitive array slot, so getPrimitiveArray must surface this rather + // than substitute a zero/false value silently. The String and Object + // overloads are allowed to keep the null element. + try (JSONEachRowFormatReader reader = readerOf(row( + "ints", Arrays.asList(1, null, 3), + "bools", Arrays.asList(Boolean.TRUE, null), + "strs", Arrays.asList("a", null, "c"), + "objs", Arrays.asList("x", null)))) { + reader.next(); + + assertThrowsRuntime(() -> reader.getIntArray("ints")); + assertThrowsRuntime(() -> reader.getIntArray(1)); + assertThrowsRuntime(() -> reader.getLongArray("ints")); + assertThrowsRuntime(() -> reader.getShortArray("ints")); + assertThrowsRuntime(() -> reader.getByteArray("ints")); + assertThrowsRuntime(() -> reader.getFloatArray("ints")); + assertThrowsRuntime(() -> reader.getDoubleArray("ints")); + assertThrowsRuntime(() -> reader.getBooleanArray("bools")); + + // The non-primitive container accessors keep nulls. + Assert.assertEquals(reader.getStringArray("strs"), new String[] {"a", null, "c"}); + Object[] objs = reader.getObjectArray("objs"); + Assert.assertEquals(objs.length, 2); + Assert.assertNull(objs[1]); + } + } + + @Test + public void testArrayAccessorsRejectNonArrayValues() throws Exception { + try (JSONEachRowFormatReader reader = readerOf(row("v", "x"))) { + reader.next(); + assertThrowsRuntime(() -> reader.getList("v")); + assertThrowsRuntime(() -> reader.getList(1)); + assertThrowsRuntime(() -> reader.getIntArray("v")); + assertThrowsRuntime(() -> reader.getIntArray(1)); + assertThrowsRuntime(() -> reader.getLongArray("v")); + assertThrowsRuntime(() -> reader.getLongArray(1)); + assertThrowsRuntime(() -> reader.getShortArray("v")); + assertThrowsRuntime(() -> reader.getShortArray(1)); + assertThrowsRuntime(() -> reader.getByteArray("v")); + assertThrowsRuntime(() -> reader.getByteArray(1)); + assertThrowsRuntime(() -> reader.getFloatArray("v")); + assertThrowsRuntime(() -> reader.getFloatArray(1)); + assertThrowsRuntime(() -> reader.getDoubleArray("v")); + assertThrowsRuntime(() -> reader.getDoubleArray(1)); + assertThrowsRuntime(() -> reader.getBooleanArray("v")); + assertThrowsRuntime(() -> reader.getBooleanArray(1)); + assertThrowsRuntime(() -> reader.getStringArray("v")); + assertThrowsRuntime(() -> reader.getStringArray(1)); + assertThrowsRuntime(() -> reader.getObjectArray("v")); + assertThrowsRuntime(() -> reader.getObjectArray(1)); + } + } + + @Test + public void testArrayAccessorsByIndex() throws Exception { + // Mirrors testArrayAccessorsCoercePrimitiveElements but addresses every + // typed array accessor through its 1-based column index. This ensures + // the index-based overloads delegate correctly through the inferred + // schema, even for arrays where the schema only records "Array" without + // a specific element type. + try (JSONEachRowFormatReader reader = readerOf(row( + "ints", Arrays.asList(1, 2, 3), + "longs", Arrays.asList(10L, 20L, 30L), + "shorts", Arrays.asList((short) 4, (short) 5), + "bytes", Arrays.asList((byte) 6, (byte) 7), + "doubles", Arrays.asList(1.5d, 2.5d), + "floats", Arrays.asList(1.0f, 2.0f), + "bools", Arrays.asList(Boolean.TRUE, Boolean.FALSE), + "strs", Arrays.asList("a", "b")))) { + reader.next(); + + Assert.assertEquals(reader.getIntArray(1), new int[] {1, 2, 3}); + Assert.assertEquals(reader.getLongArray(2), new long[] {10L, 20L, 30L}); + Assert.assertEquals(reader.getShortArray(3), new short[] {(short) 4, (short) 5}); + Assert.assertEquals(reader.getByteArray(4), new byte[] {(byte) 6, (byte) 7}); + Assert.assertEquals(reader.getDoubleArray(5), new double[] {1.5d, 2.5d}, 1e-9); + Assert.assertEquals(reader.getFloatArray(6), new float[] {1.0f, 2.0f}, 1e-6f); + Assert.assertEquals(reader.getBooleanArray(7), new boolean[] {true, false}); + Assert.assertEquals(reader.getStringArray(8), new String[] {"a", "b"}); + + // getList and getObjectArray by index, on the first column. + List list = reader.getList(1); + Assert.assertEquals(list, Arrays.asList(1, 2, 3)); + Object[] objs = reader.getObjectArray(1); + Assert.assertEquals(objs.length, 3); + } + } + + @Test + public void testGetBooleanArrayFromNumericList() throws Exception { + // Exercises the Number branch in coerceToComponent for boolean[]: the + // reader treats 0 as false and any non-zero value as true, regardless + // of whether the parser materialized the element as Integer, Long, or + // BigDecimal. + try (JSONEachRowFormatReader reader = readerOf(row( + "as_int", Arrays.asList(0, 1, 2, -1), + "as_long", Arrays.asList(0L, 5L), + "as_big_decimal", Arrays.asList(BigDecimal.ZERO, new BigDecimal("3"))))) { + reader.next(); + + Assert.assertEquals(reader.getBooleanArray("as_int"), + new boolean[] {false, true, true, true}); + Assert.assertEquals(reader.getBooleanArray("as_long"), + new boolean[] {false, true}); + Assert.assertEquals(reader.getBooleanArray("as_big_decimal"), + new boolean[] {false, true}); + } + } + + @Test + public void testGetBooleanArrayRejectsNonBooleanNonNumberElements() throws Exception { + // String elements cannot be coerced into boolean[] (we don't accept + // string-to-boolean parsing on the scalar getBoolean accessor either), + // so getBooleanArray must surface a RuntimeException sourced from the + // illegal-coerce branch in coerceToComponent. + try (JSONEachRowFormatReader reader = readerOf(row( + "v", Arrays.asList("true", "false")))) { + reader.next(); + assertThrowsRuntime(() -> reader.getBooleanArray("v")); + assertThrowsRuntime(() -> reader.getBooleanArray(1)); + } + } + + @Test + public void testArrayAccessorsCoercePrimitiveElements() throws Exception { + // Different parsers materialize numbers as different boxed types + // (Integer, Long, Double, BigDecimal). The reader must coerce each + // element to the requested primitive type without losing data. + try (JSONEachRowFormatReader reader = readerOf(row( + "ints", Arrays.asList(1, 2, 3), + "longs", Arrays.asList(10L, 20L, 30L), + "doubles", Arrays.asList(1.5d, 2.5d), + "floats", Arrays.asList(1.0f, 2.0f), + "shorts", Arrays.asList((short) 10, (short) 20), + "bytes", Arrays.asList((byte) 1, (byte) 2), + "bools", Arrays.asList(Boolean.TRUE, Boolean.FALSE, Boolean.TRUE), + "strs", Arrays.asList("a", "b"), + "big_decs", Arrays.asList(new BigDecimal("4"), new BigDecimal("5"))))) { + reader.next(); + + Assert.assertEquals(reader.getIntArray("ints"), new int[] {1, 2, 3}); + Assert.assertEquals(reader.getLongArray("longs"), new long[] {10L, 20L, 30L}); + Assert.assertEquals(reader.getDoubleArray("doubles"), new double[] {1.5d, 2.5d}, 1e-9); + Assert.assertEquals(reader.getFloatArray("floats"), new float[] {1.0f, 2.0f}, 1e-6f); + Assert.assertEquals(reader.getShortArray("shorts"), new short[] {(short) 10, (short) 20}); + Assert.assertEquals(reader.getByteArray("bytes"), new byte[] {(byte) 1, (byte) 2}); + Assert.assertEquals(reader.getBooleanArray("bools"), new boolean[] {true, false, true}); + Assert.assertEquals(reader.getStringArray("strs"), new String[] {"a", "b"}); + + // BigDecimal -> int[] requires element-level coercion. + Assert.assertEquals(reader.getIntArray("big_decs"), new int[] {4, 5}); + + // getObjectArray preserves boxed element types. + Object[] objs = reader.getObjectArray("ints"); + Assert.assertEquals(objs.length, 3); + + // getList returns the parser-native list reference. + List list = reader.getList("ints"); + Assert.assertEquals(list, Arrays.asList(1, 2, 3)); + } + } + + private static void assertThrowsRuntime(Runnable r) { + try { + r.run(); + Assert.fail("Expected RuntimeException"); + } catch (RuntimeException expected) { + // ok + } + } + + private static void assertUnsupported(Runnable r) { + try { + r.run(); + Assert.fail("Expected UnsupportedOperationException"); + } catch (UnsupportedOperationException expected) { + // ok + } + } + + @Test + public void testParseOnReadAccessorsReturnNullForNullValue() throws Exception { + // Locks in null-tolerant behavior for accessors that parse the value out + // of its textual representation. A null cell must propagate cleanly + // rather than NPE while calling toString() on the missing value. + Map r = new LinkedHashMap<>(); + r.put("u", null); + r.put("d", null); + r.put("t", null); + r.put("dt", null); + r.put("odt", null); + + try (JSONEachRowFormatReader reader = new JSONEachRowFormatReader( + new StubJsonParser(Collections.singletonList(r)))) { + reader.next(); + + Assert.assertNull(reader.getUUID("u")); + Assert.assertNull(reader.getUUID(1)); + Assert.assertNull(reader.getLocalDate("d")); + Assert.assertNull(reader.getLocalDate(2)); + Assert.assertNull(reader.getLocalTime("t")); + Assert.assertNull(reader.getLocalTime(3)); + Assert.assertNull(reader.getLocalDateTime("dt")); + Assert.assertNull(reader.getLocalDateTime(4)); + Assert.assertNull(reader.getOffsetDateTime("odt")); + Assert.assertNull(reader.getOffsetDateTime(5)); + } + } +} diff --git a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/JacksonJSONEachRowFormatReaderTests.java b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/JacksonJSONEachRowFormatReaderTests.java new file mode 100644 index 000000000..03aaab38b --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/JacksonJSONEachRowFormatReaderTests.java @@ -0,0 +1,23 @@ +package com.clickhouse.client.api.data_formats; + +import com.clickhouse.client.api.query.QueryResponse; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.io.InputStream; + +@Test(groups = {"integration"}) +public class JacksonJSONEachRowFormatReaderTests extends AbstractJSONEachRowFormatReaderTests { + + private JsonParserFactory parserFactory = new JacksonJsonParserFactory(); + + @Override + protected ClickHouseTextFormatReader createReader(QueryResponse response) throws IOException { + return new JSONEachRowFormatReader(parserFactory.createJsonParser(response.getInputStream())); + } + + @Override + protected ClickHouseTextFormatReader createReader(InputStream input) throws IOException { + return new JSONEachRowFormatReader(parserFactory.createJsonParser(input)); + } +} diff --git a/client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/JacksonJsonParserTest.java b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/JacksonJsonParserTest.java new file mode 100644 index 000000000..1ad5778bc --- /dev/null +++ b/client-v2/src/test/java/com/clickhouse/client/api/data_formats/internal/JacksonJsonParserTest.java @@ -0,0 +1,102 @@ +package com.clickhouse.client.api.data_formats.internal; + +import com.clickhouse.client.api.data_formats.JacksonJsonParserFactory; +import com.clickhouse.client.api.data_formats.JsonParser; +import com.fasterxml.jackson.core.JsonParseException; +import org.testng.Assert; +import org.testng.annotations.Test; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +@Test(groups = {"unit"}) +public class JacksonJsonParserTest { + + private JacksonJsonParserFactory parserFactory = new JacksonJsonParserFactory(); + + private static InputStream input(String json) { + return new ByteArrayInputStream(json.getBytes(StandardCharsets.UTF_8)); + } + + @Test + public void testReadsSingleRow() throws Exception { + try (JsonParser parser = parserFactory.createJsonParser( + input("{\"id\": 1, \"name\": \"a\"}"))) { + Map row = parser.nextRow(); + Assert.assertNotNull(row); + Assert.assertEquals(((Number) row.get("id")).intValue(), 1); + Assert.assertEquals(row.get("name"), "a"); + + // After the only row, EOF must be reported as null. + Assert.assertNull(parser.nextRow()); + } + } + + @Test + public void testReadsMultipleRowsSeparatedByWhitespace() throws Exception { + // JSONEachRow output is a sequence of JSON objects separated by + // newlines. Jackson skips whitespace between tokens, so the parser + // must transparently advance to each subsequent object. + String body = "{\"id\":1}\n{\"id\":2}\n {\"id\":3}\n"; + try (JsonParser parser = parserFactory.createJsonParser(input(body))) { + for (int expected : Arrays.asList(1, 2, 3)) { + Map row = parser.nextRow(); + Assert.assertNotNull(row, "row " + expected + " should not be null"); + Assert.assertEquals(((Number) row.get("id")).intValue(), expected); + } + Assert.assertNull(parser.nextRow()); + } + } + + @Test + public void testEmptyInputReturnsNull() throws Exception { + try (JsonParser parser = parserFactory.createJsonParser(input(""))) { + Assert.assertNull(parser.nextRow()); + } + } + + @Test + public void testWhitespaceOnlyInputReturnsNull() throws Exception { + try (JsonParser parser = parserFactory.createJsonParser(input(" \n\n "))) { + Assert.assertNull(parser.nextRow()); + } + } + + @Test + public void testRepeatedNextRowAfterExhaustionRemainsNull() throws Exception { + try (JsonParser parser = parserFactory.createJsonParser(input("{\"id\":1}"))) { + Assert.assertNotNull(parser.nextRow()); + Assert.assertNull(parser.nextRow()); + Assert.assertNull(parser.nextRow()); + } + } + + @Test(expectedExceptions = JsonParseException.class) + public void testMalformedInputPropagatesParseException() throws Exception { + try (JsonParser parser = parserFactory.createJsonParser(input("{not valid json"))) { + parser.nextRow(); + } + } + + @Test + public void testCloseClosesUnderlyingStream() throws Exception { + AtomicBoolean closed = new AtomicBoolean(false); + InputStream stream = new ByteArrayInputStream("{\"id\":1}".getBytes(StandardCharsets.UTF_8)) { + @Override + public void close() throws IOException { + closed.set(true); + super.close(); + } + }; + + JsonParser parser = parserFactory.createJsonParser(stream); + parser.close(); + Assert.assertTrue(closed.get(), "Underlying input stream should be closed"); + } + +} 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 2651f88e9..d9db0d0ce 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 @@ -7,6 +7,7 @@ import com.clickhouse.client.ClickHouseProtocol; import com.clickhouse.client.ClickHouseServerForTest; import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.ClientConfigProperties; import com.clickhouse.client.api.ClientException; import com.clickhouse.client.api.ServerException; import com.clickhouse.client.api.command.CommandSettings; @@ -380,6 +381,44 @@ public void testQueryJSONEachRow() throws ExecutionException, InterruptedExcepti } } + @Test(groups = {"integration"}) + public void testJsonEachRowNumberQuoteSettingsAreOptIn() throws Exception { + // Pin all three numeric quoting flags (integers, floats, decimals) to `1` regardless + // of server defaults so the test exercises the same starting state on every supported + // ClickHouse version. The server default for 64-bit integers flipped from `1` to `0` + // in 25.8 (PR #74079); explicit overrides keep the assertions stable across versions. + String sql = "SELECT toInt64(1234567890123) AS i, toFloat64(3.14) AS f, toDecimal64(1.5, 2) AS d"; + + QuerySettings settings = new QuerySettings() + .setFormat(ClickHouseFormat.JSONEachRow) + .serverSetting("output_format_json_quote_64bit_integers", "1") + .serverSetting("output_format_json_quote_64bit_floats", "1") + .serverSetting("output_format_json_quote_decimals", "1"); + try (QueryResponse response = client.query(sql, settings).get(); + BufferedReader reader = new BufferedReader(new InputStreamReader(response.getInputStream()))) { + String line = reader.readLine(); + Assert.assertTrue(line.contains("\"i\":\"1234567890123\""), "Int64 should be quoted: " + line); + Assert.assertTrue(line.contains("\"f\":\"3.14\""), "Float64 should be quoted: " + line); + // Decimal trailing-zero formatting differs across server versions; match the + // significant digits inside the quotes. + Assert.assertTrue(line.matches(".*\"d\":\"1\\.50?\".*"), "Decimal should be quoted: " + line); + } + + QuerySettings unquotedSettings = new QuerySettings() + .setFormat(ClickHouseFormat.JSONEachRow) + .serverSetting("output_format_json_quote_64bit_integers", "1") + .serverSetting("output_format_json_quote_64bit_floats", "1") + .serverSetting("output_format_json_quote_decimals", "1") + .setOption(ClientConfigProperties.JSON_DISABLE_NUMBER_QUOTING.getKey(), true); + try (QueryResponse response = client.query(sql, unquotedSettings).get(); + BufferedReader reader = new BufferedReader(new InputStreamReader(response.getInputStream()))) { + String line = reader.readLine(); + Assert.assertTrue(line.contains("\"i\":1234567890123"), "Int64 should be unquoted: " + line); + Assert.assertTrue(line.contains("\"f\":3.14"), "Float64 should be unquoted: " + line); + Assert.assertTrue(line.matches(".*\"d\":1\\.50?[,}].*"), "Decimal should be unquoted: " + line); + } + } + @DataProvider(name = "rowBinaryFormats") ClickHouseFormat[] getRowBinaryFormats() { return new ClickHouseFormat[]{ diff --git a/docs/client-v2-json-support.md b/docs/client-v2-json-support.md new file mode 100644 index 000000000..ac34027c3 --- /dev/null +++ b/docs/client-v2-json-support.md @@ -0,0 +1,644 @@ +# JSONEachRow Support in `client-v2` and `jdbc-v2` + +This document specifies the `JSONEachRow` output-format support introduced in +`client-v2` and exposed through `jdbc-v2`. It defines the public API, +configuration properties, runtime dependencies, type mapping, and current +limitations. + +## Motivation + +ClickHouse provides several JSON-oriented column types (`JSON`, `Variant`, +`Dynamic`) and structured types (`Array`, `Tuple`, `Map`). When such values +are returned through binary formats they are commonly materialized as +serialized strings, which requires every caller to embed and configure its own +JSON parser and complicates the propagation of nested objects through JDBC. + +`JSONEachRow` is the row-oriented JSON output format of ClickHouse: each row +is emitted as a self-contained JSON object separated by line breaks. With the +appropriate server settings, numeric values are emitted as JSON numbers and +nested objects are preserved without additional encoding. Supporting this +format directly in the Java clients enables: + +- materializing columns of type `JSON` as `Map` instances; +- preserving nested objects, arrays, and tuples without an additional parsing + step; +- exposing structured JSON payloads through the standard JDBC `ResultSet` + contract. + +Combining `JSONEachRow` output with a pluggable Jackson or Gson parser +provides additional advantages beyond what the format alone delivers: + +- **Streaming row parsing.** Jackson's `JsonParser` and Gson's `JsonReader` + consume the response stream incrementally. `JSONEachRowFormatReader` + materializes one row at a time, so peak memory consumption is bounded by + the size of the current row rather than by the size of the result set. +- **Reuse of an existing JSON dependency on the classpath.** Applications + that already depend on Jackson or Gson for unrelated purposes can instantiate + the matching `JsonParserFactory` and avoid contributing a second JSON library + to the runtime classpath. Only the library JARs are shared; the default + factories use their own `ObjectMapper` or `Gson` instance unless callers + provide a subclass with custom settings. +- **Choice between processors.** Jackson and Gson are selected independently + and can be swapped at deployment time. Applications may pick the processor + that best matches their existing classpath and operational constraints, + without changing application code that consumes the reader or the JDBC + `ResultSet`. + +## Scope and non-goals + +This section defines, in one place, what the JSONEachRow support is built +to do and what it intentionally leaves out. + +### The problem + +`JSONEachRow` itself is a simple line-oriented format, but mapping its +contents to Java values is not. JSON has fewer primitive types than +ClickHouse, applications disagree on how large integers, decimals, dates, +and other domain values should surface in Java, and the two mainstream +JSON libraries (Jackson and Gson) make different default choices for many +of these mappings. Bundling a single fixed JSON-to-Java mapping in the +client would force every application into one library's conventions and +would re-implement work the application's existing JSON dependency +already does. + +At the same time, application code that already consumes the binary +readers should not have to fork into a separate read loop just because +the result is in a text format. + +### Goals + +1. Provide a row-by-row reader over `JSONEachRow` that delegates parsing + and per-token type decisions to whichever JSON library the application + already runs. +2. Let applications customize that mapping through the chosen library + (Jackson modules / features, Gson `TypeAdapter`s, number policies, + custom domain types) by extending the bundled factory and overriding + its single protected hook. +3. Expose a single accessor surface (`ClickHouseFormatReader`) shared + with the binary readers so caller code can be format-agnostic. +4. Keep the parser SPI (`JsonParser`, `JsonParserFactory`) narrow and + library-agnostic so applications with parser requirements outside + Jackson or Gson can plug in their own implementation without changes + to `client-v2`. + +### What is covered + +- Streaming row parsing of `JSONEachRow` through a pluggable + `JsonParserFactory`. Each row materializes as `Map`; + leaf value types are whatever the configured library chose. +- Bundled `JacksonJsonParserFactory` and `GsonJsonParserFactory` with + sensible defaults and one protected hook each for customization + (`createMapper()` and `customize(GsonBuilder)`). +- A common reader hierarchy — `ClickHouseFormatReader` for the shared + accessor surface, `ClickHouseTextFormatReader` as the text-format + family marker, and `JSONEachRowFormatReader` as the concrete reader. +- An opt-in server-setting bundle that disables JSON number quoting for + `JSONEachRow` requests (see + [JSON number output settings](#json-number-output-settings)). +- JDBC integration: `FORMAT JSONEachRow` is accepted by + `Statement.executeQuery(...)` in `jdbc-v2`, with parser-factory + selection through the `jdbc_json_parser_factory` connection property. + +### What is not covered + +- **Library-native row objects.** The reader does not expose Jackson's + `JsonNode` or Gson's `JsonElement` for the current row, and there is + no `currentRowAsObject` / `currentRowAsString` accessor on the reader + interface. The `Map` view is the contract. + Applications that need the library-native representation should + implement their own `JsonParserFactory` / `JsonParser` that retains + the native object internally and exposes it through their own + accessor — the SPI is intentionally narrow so this kind of extension + lives in application code rather than in `client-v2`. +- **Uniform cross-library semantics.** The reader does not normalize Java + type choices across libraries. Jackson and Gson disagree on, for + example, whether an unquoted JSON integer larger than `2^53` is + materialized as an integral `Number` or as a `Double`. The reader + respects whatever the configured library chose; consistent behavior + across processors is the application's responsibility, via factory + customization or post-processing. +- **A built-in JSON tokenizer.** The reader does not include its own + parser. At least one of the bundled libraries (Jackson or Gson) must + be on the runtime classpath, or the application must supply its own + `JsonParserFactory`, for `JSONEachRow` support to function. +- **Full accessor parity with the binary readers.** Several advanced + accessors (temporal/inet/geo/bitmap families) are not implemented for + `JSONEachRow` and throw `UnsupportedOperationException`. See + [Typed accessors](#typed-accessors) for the current list. Callers + needing these conversions should read the value through + `readValue(...)` / `getList(...)` / `getString(...)` and convert at + the application boundary, or use the binary default format where + these accessors are natively supported. + +## Summary of changes + +`client-v2`: + +- Introduces a common `com.clickhouse.client.api.data_formats.ClickHouseFormatReader` + interface that declares all row navigation, schema access, and typed + accessors. The pre-existing `ClickHouseBinaryFormatReader` becomes a + format-family sub-interface for binary output formats and inherits its + full method set unchanged from `ClickHouseFormatReader`. +- Adds `com.clickhouse.client.api.data_formats.ClickHouseTextFormatReader`, + a sibling sub-interface for text output formats. +- Adds `com.clickhouse.client.api.data_formats.JSONEachRowFormatReader`, + which implements `ClickHouseTextFormatReader` over a streaming JSON + parser. +- `Client.newBinaryFormatReader(...)` continues to construct the binary + readers (`Native`, `RowBinary`, `RowBinaryWithNames`, + `RowBinaryWithNamesAndTypes`) and rejects text formats with + `IllegalArgumentException`. `JSONEachRow` callers construct + `JSONEachRowFormatReader` directly from a `JsonParser`. +- Introduces a JSON parser SPI under + `com.clickhouse.client.api.data_formats`, consisting of `JsonParser`, + `JsonParserFactory`, `JacksonJsonParserFactory`, and + `GsonJsonParserFactory`. +- Adds an opt-in client flag for `JSONEachRow` requests that asks ClickHouse + to emit large integers, floats, and decimals as plain JSON numbers (see + [JSON number output settings](#json-number-output-settings)). +- Declares Jackson and Gson as `provided` Maven dependencies, so that + applications must include the chosen processor on their own classpath. + +`jdbc-v2`: + +- Modifies `StatementImpl.executeQuery(...)` to accept `JSONEachRow` as a + valid output format. All other text formats remain unsupported. +- Adds `DriverProperties.JSON_PARSER_FACTORY` + (key `jdbc_json_parser_factory`) for selecting the `JsonParserFactory` + implementation by fully-qualified class name. +- Declares Jackson and Gson as `provided` dependencies, consistent with + `client-v2`. + +Two runnable examples are included in the repository: +`examples/client-v2-json-processors` and `examples/jdbc-v2-json-processors`. + +## Public API + +### `ClickHouseFormatReader`, `ClickHouseBinaryFormatReader`, `ClickHouseTextFormatReader` + +```java +package com.clickhouse.client.api.data_formats; + +public interface ClickHouseFormatReader extends AutoCloseable { ... } + +public interface ClickHouseBinaryFormatReader extends ClickHouseFormatReader { } + +public interface ClickHouseTextFormatReader extends ClickHouseFormatReader { } +``` + +`ClickHouseFormatReader` is the common contract for row-by-row format +readers regardless of the underlying wire encoding. The two sub-interfaces +specialize that contract by output-format family: callers receive a +`ClickHouseBinaryFormatReader` when the response is in a binary format and a +`ClickHouseTextFormatReader` when it is in a text format. All accessor +methods declared today live on the common parent; future format-specific +extensions are expected to be added on the corresponding sub-interface +without changing the shared surface, so code written against +`ClickHouseBinaryFormatReader` continues to compile against the same +inherited methods. + +### `JSONEachRowFormatReader` + +```java +package com.clickhouse.client.api.data_formats; + +public class JSONEachRowFormatReader implements ClickHouseTextFormatReader { ... } +``` + +The reader is instantiated from an `InputStream`-backed `JsonParser`. Callers +usually create that parser through `JacksonJsonParserFactory`, +`GsonJsonParserFactory`, or an application subclass of one of those factories. + +`JSONEachRow` is a text format, so the reader implements +`ClickHouseTextFormatReader`. Callers that need to handle both binary and +text readers uniformly can program against `ClickHouseFormatReader`. + +### `JsonParser` SPI + +```java +package com.clickhouse.client.api.data_formats; + +public interface JsonParser extends AutoCloseable { + Map nextRow() throws Exception; +} +``` + +Two factories are provided: + +- `JacksonJsonParserFactory` uses `com.fasterxml.jackson.core` and + `com.fasterxml.jackson.databind` to stream JSON objects. +- `GsonJsonParserFactory` uses `com.google.gson` with a lenient `JsonReader`, + which accepts a sequence of top-level JSON objects separated by whitespace, + as produced by `JSONEachRow`. + +`JsonParserFactory.createJsonParser(InputStream)` creates a parser for each +response stream. The factory may hold reusable parser configuration, but the +returned `JsonParser` is request-scoped and owns the stream reader it creates. + +The shipped implementations construct their own `ObjectMapper` and `Gson` +instances with default settings. To customize the underlying library +(Jackson modules, feature flags, Gson `TypeAdapter`s, number policies, etc.) +extend the corresponding factory and override its protected hook: + +- `JacksonJsonParserFactory` exposes `protected ObjectMapper createMapper()`. + Override it to return a fully configured `ObjectMapper`; the factory uses + the returned instance for all subsequent row parsing. +- `GsonJsonParserFactory` exposes `protected void customize(GsonBuilder + builder)`. Override it to configure the `GsonBuilder` before the factory + applies `setLenient()` and calls `build()`. + +Customization that does not need to influence the underlying parser can +still be performed on the caller side, after the row has been materialized +as `Map`. + +### JDBC parser factory property + +`jdbc-v2` selects the parser factory through +`DriverProperties.JSON_PARSER_FACTORY`: + +| Property key | Value | +| ------------------------- | ----- | +| `jdbc_json_parser_factory` | Fully-qualified class name implementing `JsonParserFactory` | + +The named class is loaded reflectively when the connection is created and must +have a public no-argument constructor. There is no equivalent `client-v2` +configuration key; direct client users pass a factory instance to their own +reader construction code. + +### JSON number quoting flag + +`client-v2` can opt in to numeric JSON output settings through +`ClientConfigProperties.JSON_DISABLE_NUMBER_QUOTING`: + +| Property key | Default | Effect | +| --------------------------------------------- | ------- | ------ | +| `json_disable_number_quoting` | `false` | When `true` and the resolved request format is `JSONEachRow`, sets `output_format_json_quote_64bit_integers=0`, `output_format_json_quote_64bit_floats=0`, and `output_format_json_quote_decimals=0` for that request. | + +The flag can be set on the client builder or on a specific `QuerySettings` +instance. It does not change `output_format_json_quote_denormals`. + +## Runtime dependencies + +`client-v2` and `jdbc-v2` declare the JSON libraries with `provided` scope, +so that they are not contributed to the runtime classpath of applications +that do not require them. Applications must add the JSON library used by the +selected factory to their runtime classpath: + +- Jackson — `com.fasterxml.jackson.core:jackson-databind`, + `com.fasterxml.jackson.core:jackson-core`, + `com.fasterxml.jackson.core:jackson-annotations` + (required when using `JacksonJsonParserFactory`); or +- Gson — `com.google.code.gson:gson` + (required when using `GsonJsonParserFactory`). + +The repository builds against Jackson `2.17.2` and Gson `2.10.1`. The parser +implementations rely only on the streaming token API and a single +`Map` materialization call, so other recent versions of +either library are expected to be compatible. + +When the selected JSON library is not present on the classpath, construction +or first use of the corresponding factory fails with the dependency-loading +error raised by the JVM or the JSON library. + +## Usage in `client-v2` + +```java +Client client = new Client.Builder() + .addEndpoint("http://localhost:8123") + .setUsername("default") + .setPassword("") + .setDefaultDatabase("default") + .serverSetting("allow_experimental_json_type", "1") + .build(); + +JsonParserFactory parserFactory = new JacksonJsonParserFactory(); + +QuerySettings settings = new QuerySettings() + .setFormat(ClickHouseFormat.JSONEachRow) + .setOption(ClientConfigProperties.JSON_DISABLE_NUMBER_QUOTING.getKey(), true); + +try (QueryResponse response = client.query( + "SELECT id, name, active, score, payload FROM events ORDER BY id", + settings).get(); + ClickHouseTextFormatReader reader = new JSONEachRowFormatReader( + parserFactory.createJsonParser(response.getInputStream()))) { + while (reader.next() != null) { + int id = reader.getInteger("id"); + String name = reader.getString("name"); + boolean active = reader.getBoolean("active"); + double score = reader.getDouble("score"); + Map payload = reader.readValue("payload"); // JSON column + // ... + } +} +``` + +Notes: + +- Set `ClickHouseFormat.JSONEachRow` in `QuerySettings`. Do not rely on an SQL + `FORMAT JSONEachRow` clause for direct `client-v2` examples when you also + want client-side JSON number output settings, because those settings are + applied only when the request settings identify the format as `JSONEachRow`. +- `client.newBinaryFormatReader(response)` continues to return a + `ClickHouseBinaryFormatReader` for binary output formats and rejects text + formats such as `JSONEachRow` with `IllegalArgumentException`. Callers that + need to handle both can program against the shared `ClickHouseFormatReader` + parent interface. +- `Map` is the canonical materialization for JSON columns + and for the row itself, as produced by the selected library. JSON arrays + are returned as `List`; nested JSON objects are returned as nested + `Map` instances. The exact Java types of leaf values are + whatever Jackson or Gson chose during parsing. + +## Usage in `jdbc-v2` + +The output format is selected by appending `FORMAT JSONEachRow` to the SQL +statement. The driver does not rewrite the SQL and does not apply a default +format on the caller's behalf. + +```java +Properties props = new Properties(); +props.setProperty("user", "default"); +props.setProperty("password", ""); +props.setProperty(DriverProperties.JSON_PARSER_FACTORY.getKey(), + JacksonJsonParserFactory.class.getName()); +// The JSON column type is experimental on the server side. +props.setProperty(ClientConfigProperties.serverSetting("allow_experimental_json_type"), "1"); +props.setProperty(ClientConfigProperties.serverSetting("output_format_json_quote_64bit_integers"), "0"); +props.setProperty(ClientConfigProperties.serverSetting("output_format_json_quote_64bit_floats"), "0"); +props.setProperty(ClientConfigProperties.serverSetting("output_format_json_quote_decimals"), "0"); + +try (Connection conn = DriverManager.getConnection( + "jdbc:clickhouse://localhost:8123/default", props); + Statement stmt = conn.createStatement(); + ResultSet rs = stmt.executeQuery( + "SELECT id, name, active, score, payload " + + "FROM events ORDER BY id FORMAT JSONEachRow")) { + + while (rs.next()) { + int id = rs.getInt("id"); + String name = rs.getString("name"); + boolean active = rs.getBoolean("active"); + double score = rs.getDouble("score"); + Object payload = rs.getObject("payload"); // Map / List / scalar + } +} +``` + +Behavior: + +- When `FORMAT JSONEachRow` is not specified, `jdbc-v2` continues to use the + binary default. `StatementImpl` accepts only `RowBinaryWithNamesAndTypes` + and `JSONEachRow` as output formats; any other text format causes + `SQLException("Only RowBinaryWithNameAndTypes and JSONEachRow are supported + for output format. ...")` to be thrown. +- `ResultSet.getObject(...)` returns parser-native `Map`, `List`, and scalar + values without an additional string round-trip. JSON arrays are returned as + the `List` implementation produced by the selected JSON library. Because + `JSONEachRow` has no array element metadata, `ResultSet.getArray(...)` is + not supported for these inferred JSON arrays. +- Temporal typed JDBC accessors follow the current `JSONEachRowFormatReader` + text-accessor support. `ResultSet.getString(...)` can be used to read the + server-formatted temporal text, but `getTimestamp(...)`, + `getObject(..., Timestamp.class)`, and related temporal conversions are not + guaranteed for `FORMAT JSONEachRow` result sets. Use the binary default + format when JDBC temporal typed accessors are required, or read the value as + a string/object and convert it in application code. +- The JSON processor is selected at the connection level through the + `jdbc_json_parser_factory` driver property. It cannot be changed per + statement, in line with the lifecycle of other connection options. +- Because JDBC selects `JSONEachRow` through SQL text, set the JSON output + server settings explicitly as connection properties when integer or decimal + numeric accessors are used. + +## JSON number output settings + +`Client.applyFormatSpecificSettings(...)` runs after request settings have +been merged and after the request format has been resolved. When the format +is `JSONEachRow` and +`ClientConfigProperties.JSON_DISABLE_NUMBER_QUOTING` is enabled, the +following server-side settings are set to `0` for the request: + +| Setting | Value | Rationale | +| ----------------------------------------- | ------------ | -------------------------------------------------------------------------- | +| `output_format_json_quote_64bit_integers` | `0` | Emits `Int64` and `UInt64` as JSON numbers rather than quoted strings. | +| `output_format_json_quote_64bit_floats` | `0` | Emits 64-bit floating-point values as JSON numbers. | +| `output_format_json_quote_decimals` | `0` | Emits decimals as JSON numbers, allowing materialization as `BigDecimal` or `Double`. | + +These overrides are scoped to the individual request and apply only when both +conditions are true: the request format in `QuerySettings` is `JSONEachRow`, +and `json_disable_number_quoting` is enabled through the client +or request settings. Explicit server settings are otherwise preserved. + +Denormal floating-point values (`NaN`, `Inf`, `-Inf`) are not yet handled by +the built-in JSON reader. The client does not set +`output_format_json_quote_denormals`; keep the server default or set +`output_format_json_quote_denormals=1` so these values are quoted, then handle +them as strings at the application boundary. + +JDBC callers that use SQL `FORMAT JSONEachRow` should set the same numeric +server settings explicitly through connection properties when integer or +decimal numeric accessors are used. + +## Row parsing, schema, and typed accessors + +### Row parsing is delegated to the chosen library + +The reader does not implement its own JSON parser. Each row is materialized +by the configured library: + +- the Jackson backend calls `ObjectMapper.readValue(parser, Map.class)` on + Jackson's streaming `JsonParser`; +- the Gson backend calls `gson.fromJson(reader, TypeToken>)` + on a lenient Gson `JsonReader`. + +The result of each call is a `Map` whose values have the +runtime Java types chosen by the library for the parsed JSON tokens — +typically `Number` (for example `Integer`, `Long`, `Double`, `BigDecimal`), +`Boolean`, `String`, `List` for JSON arrays, nested +`Map` for JSON objects, and `null` for JSON `null`. Numeric +representation, widening rules, handling of large integers, and any other +JSON-to-Java decisions are governed entirely by the library. The reader +neither inspects raw JSON tokens nor overrides the library's parsing +behavior. + +### Integer precision with Gson + +ClickHouse `Int64` and `UInt64` values can exceed the exactly representable +integer range of a JSON floating-point number. When +`json_disable_number_quoting` is enabled, the client asks +ClickHouse to emit them as JSON numbers for `JSONEachRow`, so the selected +JSON library's number materialization policy matters. + +Jackson's default `Map.class` materialization keeps ordinary integer tokens as +integer `Number` implementations. Gson's default `Map` +materialization can surface JSON numbers as floating-point values, which may +round integers larger than `2^53` before `getLong(...)` sees them. + +If you use Gson and need integer precision, provide a custom factory that +configures Gson's object number strategy: + +```java +public final class PreciseGsonJsonParserFactory extends GsonJsonParserFactory { + @Override + protected void customize(GsonBuilder builder) { + builder.setObjectToNumberStrategy(com.google.gson.ToNumberPolicy.LONG_OR_DOUBLE); + } +} +``` + +Use that factory directly with `client-v2`: + +```java +JsonParserFactory parserFactory = new PreciseGsonJsonParserFactory(); +``` + +For JDBC, put the factory class name in the connection properties: + +```java +props.setProperty(DriverProperties.JSON_PARSER_FACTORY.getKey(), + PreciseGsonJsonParserFactory.class.getName()); +``` + +`ToNumberPolicy.LONG_OR_DOUBLE` preserves values that fit in `long` as +`Long`. If exact decimal representation is more important than returning +`Long` for integer tokens, use `ToNumberPolicy.BIG_DECIMAL` and convert +explicitly at the application boundary. + +### Minimal schema discovery + +`JSONEachRow` does not include a schema header. To populate a minimal +`TableSchema` for the typed accessors, the reader inspects the **Java +types** of the first row's values, after the library has produced them, and +maps each to a `ClickHouseDataType`: + +| Java type produced by the library | Inferred ClickHouse type | +| ---------------------------------------------------------------------- | ------------------------ | +| `Integer` | `Int32` | +| `Long` | `Int64` | +| `BigInteger` | `Int256` | +| `Float` | `Float32` | +| `Double` | `Float64` | +| `BigDecimal` | `Decimal` | +| `Boolean` | `Bool` | +| `List` or Java array | `Array` | +| `Map` | `Map` | +| Any other value (`String`, `null`, unsupported number subtypes, ...) | `String` | + +Implications: + +- Schema discovery is performed once, on the first row. Empty result sets + produce a schema with no columns. +- Column names are taken verbatim from the JSON keys of the first row, in + iteration order. +- The discovered schema is intended only to support the typed accessors + (`getInteger`, `getString`, and so on). Server-side column metadata such + as precision, nullability, and codec is not reconstructed. +- Whether a JSON number is materialized as `Integer`, `Long`, `Double`, or + `BigDecimal` is a property of the chosen library, not of the reader. + Applications that need a specific numeric representation should select + the processor whose default behavior matches their expectations. + +### Typed accessors + +The typed accessors declared on the read interface are implemented as +follows: + +| Accessor | Behavior | +| --------------------------------------------- | ------------------------------------------------------------- | +| `readValue` / `next` | Returns the row as a `Map`. | +| `getString` | Returns `Object#toString()` of the JSON value, or `null`. | +| `getByte` / `getShort` / `getInteger` / `getLong` / `getFloat` / `getDouble` | Casts through `Number`. | +| `getBoolean` | Accepts `Boolean`, non-zero `Number`, or parses a string. | +| `getBigInteger` / `getBigDecimal` | Routes through `BigDecimal(String)`. | +| `getLocalDate` / `getLocalTime` / `getLocalDateTime` / `getOffsetDateTime` | Uses the corresponding `parse(...)` method on the string value. | +| `getUUID` | Uses `UUID.fromString(...)` on the string value. | +| `getList` | Returns the JSON array as `List`. | +| `getTuple` | Returns the row value cast to `Object[]`. | +| `getEnum8` / `getEnum16` | Delegates to `getByte` / `getShort`. | + +Accessor limitations to keep in mind: + +- `getTuple(...)` does not adapt parser-native `List` or `Map` values. Since + JSON arrays are usually materialized as `List` and JSON objects as `Map`, + callers should use `readValue(...)` for tuple-like JSON values and convert + them explicitly. + +The following accessors are not supported by the current implementation and +throw `UnsupportedOperationException`: + +- `getInstant`, `getZonedDateTime`, `getDuration`, `getTemporalAmount`; +- `getInet4Address`, `getInet6Address`; +- `getGeoPoint`, `getGeoRing`, `getGeoPolygon`, `getGeoMultiPolygon`; +- the typed array accessors `getByteArray`, `getIntArray`, `getLongArray`, + `getFloatArray`, `getDoubleArray`, `getBooleanArray`, `getShortArray`, + `getStringArray`, `getObjectArray`; +- `getClickHouseBitmap`. + +For these types, callers should obtain the parsed value through +`readValue(...)` or `getList(...)` and convert it explicitly. + +## Streaming and lifetime + +- `JacksonJsonParserFactory` and `GsonJsonParserFactory` delegate parsing to the + underlying library and consume one row at a time from the response + `InputStream`. Memory consumption is proportional to the size of the + current row and is independent of the size of the result set. +- `JSONEachRowFormatReader` reads the first row eagerly during construction + in order to inspect the Java types of its values and populate the minimal + `TableSchema` described above. For empty result sets, the reader exposes + an empty `TableSchema`, and `hasNext()` returns `false`. +- `close()` is propagated to the underlying parser, which closes the input + stream it owns. Callers are responsible for closing the originating + `QueryResponse` (or JDBC `ResultSet`). + +## Compatibility considerations + +- The parser factory classes are additive. Applications that do not request + `JSONEachRow` and do not instantiate or configure a parser factory are + unaffected. +- The default request format is unchanged. The existing binary readers + (`Native`, `RowBinary`, `RowBinaryWithNames`, `RowBinaryWithNamesAndTypes`) + retain their previous behavior. +- The reader hierarchy now distinguishes binary and text formats: + `ClickHouseBinaryFormatReader` and `ClickHouseTextFormatReader` are sibling + sub-interfaces of the new `ClickHouseFormatReader`. The accessor surface is + unchanged; callers that hold a `ClickHouseBinaryFormatReader` reference for + binary formats are unaffected. `Client.newBinaryFormatReader(...)` rejects + `JSONEachRow` with `IllegalArgumentException`; construct + `JSONEachRowFormatReader` directly for JSONEachRow streams. +- Jackson and Gson are now declared with `provided` scope in `client-v2` and + `jdbc-v2`. Applications that previously inherited Jackson transitively from + these modules in `test` scope must declare the chosen processor explicitly + on their runtime classpath. +- `jdbc_json_parser_factory` is a new JDBC driver property and is only needed + by connections that execute `FORMAT JSONEachRow` queries. + +## Examples + +Two runnable Gradle examples are provided under `examples/`: + +- `examples/client-v2-json-processors` exercises the `client-v2` API + directly, switching between Jackson and Gson factories against a shared sample + table that contains primitive columns and one `payload JSON` column. + Entry point: `ClientV2JsonProcessorsExample`. +- `examples/jdbc-v2-json-processors` performs the same flow through the JDBC + driver, with `FORMAT JSONEachRow` appended to the `SELECT` statement and + parser factory selection applied through connection properties. Entry point: + `JdbcV2JsonProcessorsExample`. + +Both examples include a sample dataset under +`src/main/resources/sample_data.csv` and require a running ClickHouse server +with `allow_experimental_json_type=1`. + +## Tests + +- `client-v2/src/test/java/com/clickhouse/client/api/data_formats/AbstractJSONEachRowFormatReaderTests.java` + defines a parameterized integration test executed for both processors via + the subclasses `JacksonJSONEachRowFormatReaderTests` and + `GsonJSONEachRowFormatReaderTests`. The suite covers basic parsing, schema + inference, primitive type accessors, and empty result sets. +- `jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java` adds + `testJSONEachRowFormat`, which exercises + `Statement.executeQuery("... FORMAT JSONEachRow")` through the JDBC driver + against both parser factories. diff --git a/docs/features.md b/docs/features.md index be63e9f99..c69abe251 100644 --- a/docs/features.md +++ b/docs/features.md @@ -15,6 +15,7 @@ This document lists stable, user-visible behavior in `client-v2` and `jdbc-v2` t - Parameterized SQL: Accepts named query parameters and can send them through supported HTTP request encodings. - Result materialization helpers: Provides streaming `Records`, generic row access, and convenience APIs that materialize all rows into generic records or typed POJOs. - Binary format readers: Reads ClickHouse binary result formats including `Native`, `RowBinary`, `RowBinaryWithNames`, and `RowBinaryWithNamesAndTypes`. +- JSONEachRow text reader: Can stream `JSONEachRow` responses through a caller-supplied `JsonParser`, with Jackson and Gson parser factory implementations available as optional classpath dependencies, and infers a best-effort schema from the first row. - Data type conversion: Maps ClickHouse types to Java values for binary reads, POJO binding, and SQL parameter formatting, including date/time handling. - Geometry type support: For ClickHouse `25.11+`, where `Geometry` changed from a string alias to `Variant(Point, Ring, LineString, MultiLineString, Polygon, MultiPolygon)`, the client reads and writes `Geometry` values through generic records, binary readers, POJO binding, and SQL parameter formatting, using Java array dimensionality to represent the geometry shape. - Insert APIs: Supports inserting registered POJOs, raw streams, and callback-driven writers, with optional column lists and format selection. @@ -41,6 +42,8 @@ Compatibility-sensitive traits: - `Geometry` handling is shape-sensitive: supported values are 1D through 4D Java arrays representing the nested geometry variants, and unsupported shapes or non-array values are rejected during serialization. - `Geometry` write inference is dimension-based rather than fully type-specific: point, ring/line string, polygon/multi-line string, and multi-polygon are selected from array depth, so writing `Geometry` cannot currently distinguish `Ring` from `LineString` or `Polygon` from `MultiLineString`. - Session precedence is part of the contract: client session defaults apply to each request, operation settings may override them, and only the client `session_id` is mutable at runtime while other client session properties remain fixed for the lifetime of the client. +- JSONEachRow reading depends on the selected parser factory and request format settings: parser materialization determines Java value types, the reader infers minimal schema from the first row, and JSON number server settings are applied only when `QuerySettings` resolves to `ClickHouseFormat.JSONEachRow` and `json_disable_number_quoting` is enabled. +- JSONEachRow schema inference is intentionally best-effort: scalar values use Java-to-ClickHouse type mappings, while JSON arrays and objects are identified structurally as `Array` and `Map`. For arrays, maps, and some nested or ambiguous values, the inferred type may not include the most specific element, key, value, or nested ClickHouse type. ## `jdbc-v2` @@ -59,7 +62,7 @@ Compatibility-sensitive traits: - Prepared statements: Supports `?` parameters through client-side SQL rendering and validates that all parameters are bound before execution. - SQL parsing and classification: Classifies SQL to distinguish queries, updates, inserts, `USE`, and role-changing statements, with selectable parser backends. - JDBC escape processing: Translates supported JDBC escape syntax for dates, timestamps, and functions before execution. -- Result set streaming: Streams result sets from ClickHouse binary formats, enforces max-row limits, and manages result-set lifecycle correctly. +- Result set streaming: Streams result sets from ClickHouse binary formats and `FORMAT JSONEachRow`, enforces max-row limits, and manages result-set lifecycle correctly. - Result-set metadata: Exposes JDBC `ResultSetMetaData` backed by ClickHouse column schema. - Database metadata: Implements JDBC `DatabaseMetaData` for ClickHouse catalogs, schemas, tables, columns, and related capability reporting. - Parameter metadata: Reports prepared-statement parameter counts. @@ -80,6 +83,7 @@ Compatibility-sensitive traits: - String-like ClickHouse values have stable JDBC expectations: `String`, `FixedString`, and `Enum` values are returned as strings, while `UUID` is available both as `getString()` and `getObject(..., UUID.class)`. - `Geometry` has a stable JDBC mapping: metadata reports SQL type `ARRAY` with type name `Geometry`, read paths return nested Java arrays rather than custom wrappers, and write paths depend on the caller preserving the intended point/array nesting shape. - JDBC `Geometry` writes share the same ambiguity as the client serializer: variant selection is inferred from nesting depth, so `Ring` versus `LineString` and `Polygon` versus `MultiLineString` are not currently distinguishable when writing through the generic `Geometry` path. +- JDBC `FORMAT JSONEachRow` support is opt-in through the `jdbc_json_parser_factory` driver property, whose value must be a fully-qualified `JsonParserFactory` class name with a public no-argument constructor; JSONEachRow numeric and structured value behavior follows the selected parser and configured server output settings. Inferred JSON arrays are returned from `ResultSet.getObject(...)` as parser-native `List` values rather than JDBC `Array` values because JSONEachRow does not include element metadata. JDBC temporal typed accessors such as `getTimestamp(...)` are not guaranteed for JSONEachRow result sets; callers that need stable JDBC temporal conversions should use the binary default format or perform application-level conversion from string/object values. - Binary parameters passed through `setBytes()` are encoded as ClickHouse `unhex(...)` expressions rather than text literals; empty byte arrays map to an empty string expression. - Stream and reader setters (`setAsciiStream`, `setUnicodeStream`, `setBinaryStream`, `setCharacterStream`, `setNCharacterStream`) are treated as text input encoded with the same string-escaping rules, including length-based truncation when a length is supplied. - `getString()` formatting for temporal values is stable output: `Date` uses `yyyy-MM-dd`, `DateTime` uses `yyyy-MM-dd HH:mm:ss`, and `DateTime64` preserves fractional precision, all interpreted in server timezone context where applicable. diff --git a/examples/client-v2-json-processors/.gitignore b/examples/client-v2-json-processors/.gitignore new file mode 100644 index 000000000..f8b92c3aa --- /dev/null +++ b/examples/client-v2-json-processors/.gitignore @@ -0,0 +1,2 @@ +.gradle +build diff --git a/examples/client-v2-json-processors/README.md b/examples/client-v2-json-processors/README.md new file mode 100644 index 000000000..ff8360664 --- /dev/null +++ b/examples/client-v2-json-processors/README.md @@ -0,0 +1,138 @@ +# Client V2 JSON Processors Example + +## Overview + +This standalone example shows how to consume a `JSONEachRow` response with the +`client-v2` `JSONEachRowFormatReader` and a `JsonParserFactory`. The two +factories shipped with the client are the customization points: + +- `JacksonJsonParserFactory` exposes a `protected ObjectMapper createMapper()` + hook — override it to return a fully configured `ObjectMapper` (modules, + feature flags, custom deserializers, etc.). +- `GsonJsonParserFactory` exposes a `protected void customize(GsonBuilder)` + hook — override it to configure the `GsonBuilder` (number policy, type + adapters, etc.). The factory still applies `setLenient()` on its own + afterwards, which is required for the stream-of-objects shape of + `JSONEachRow`. + +The example is structured as a small component: + +- `ClientV2JsonProcessorsExample(Client client)` holds the shared `Client` and + exposes regular instance methods (`recreateTable()`, `loadSampleData()`, + `readAll(label, factory)`, `run()`), so the class can be copied as-is into + another project and have its individual methods invoked. +- Sample rows are kept in a plain `Object[][]` constant, separate from the + SQL, so the read path stays focused on the parser factory. +- Two small subclasses, `CustomJacksonParserFactory` and + `CustomGsonParserFactory`, demonstrate the protected-hook customization. + Both also implement a tiny `PayloadConverter` interface defined inside the + example: their configured `ObjectMapper` / `Gson` is reused to convert the + raw `payload` `Map` produced by the underlying library into a typed + `Payload` POJO. The default factories do not implement the interface, so + `readAll(...)` logs the raw map for them — making the contrast between the + default behavior and the customized behavior visible in the output. + +Each read call in `run()` follows the same three-step shape: + +1. **Create the factory** — `new JacksonJsonParserFactory()` / + `new GsonJsonParserFactory()` for defaults, or an instance of a custom + subclass. +2. **Customize if needed** — only inside the subclass, by overriding the + protected hook. +3. **Execute** — `readAll(label, factory)` runs the `SELECT` and feeds the + response stream through + `new JSONEachRowFormatReader(factory.createJsonParser(...))`. + +The client example selects the output format with +`new QuerySettings().setFormat(ClickHouseFormat.JSONEachRow)`. Use that form +instead of appending `FORMAT JSONEachRow` to the SQL when calling `client-v2` +directly when you enable client-side JSON number output settings, because +those settings depend on the request format. + +## Integer Precision + +ClickHouse 64-bit integers can be larger than the exact integer range of a +JSON floating-point number. Jackson's default map materialization preserves +ordinary integer tokens as integer `Number` values. Gson's default +`Map` materialization may surface numbers as floating-point +values, which can round large integers before `getLong(...)` sees them. + +For Gson, extend `GsonJsonParserFactory` and configure the object number +strategy: + +```java +public final class PreciseGsonJsonParserFactory extends GsonJsonParserFactory { + @Override + protected void customize(GsonBuilder builder) { + builder.setObjectToNumberStrategy(com.google.gson.ToNumberPolicy.LONG_OR_DOUBLE); + } +} +``` + +The included `CustomGsonParserFactory` uses this pattern. Use +`ToNumberPolicy.BIG_DECIMAL` instead when exact decimal representation matters +more than receiving integer tokens as `Long`. + +## Requirements + +- JDK 17 or newer +- A running ClickHouse server reachable from the machine running the example +- A locally installed `client-v2` snapshot from this repository + +## How to Run + +From this directory: + +```shell +gradle run +``` + +Connection properties can be supplied as system properties: + +- `-DchEndpoint` — endpoint to connect to (default: `http://localhost:8123`) +- `-DchUser` — ClickHouse user name (default: `default`) +- `-DchPassword` — ClickHouse user password (default: empty) +- `-DchDatabase` — ClickHouse database name (default: `default`) + +Example with custom connection properties: + +```shell +gradle run \ + -DchEndpoint=http://localhost:8123 \ + -DchUser=default \ + -DchPassword= \ + -DchDatabase=default +``` + +## Executable Example + +`com.clickhouse.examples.client_v2.json_processors.ClientV2JsonProcessorsExample` + +Steps performed by `run()`: + +1. `recreateTable()` — drops and re-creates `client_v2_json_processors_example` + with primitive columns and one `payload JSON` column. +2. `loadSampleData()` — inserts the rows from the `SAMPLE_ROWS` array as a + single batched `INSERT`. +3. `readAll(...)` is invoked four times, each time with a different + `JsonParserFactory`: + - default `JacksonJsonParserFactory`; + - `CustomJacksonParserFactory`, which overrides `createMapper()` to + tolerate unknown properties and preserve big integers and decimals + exactly, and implements `PayloadConverter` to convert each row's + `payload` `Map` into a `Payload` POJO via + `ObjectMapper.convertValue(...)`; + - default `GsonJsonParserFactory`; + - `CustomGsonParserFactory`, which overrides `customize(GsonBuilder)` to + use a `LONG_OR_DOUBLE` number policy and disable HTML escaping, and + implements `PayloadConverter` to convert each row's `payload` `Map` + into a `Payload` POJO via `gson.fromJson(gson.toJsonTree(...))`. + +Logged rows include the payload value's runtime class so the difference +between the default factories (which surface a `LinkedHashMap` / +`LinkedTreeMap`) and the customized factories (which surface a `Payload`) +shows up directly in the output. + +The build keeps both `jackson-databind` and `gson` on the classpath so the +example can switch between processors at runtime. Production applications +only need to keep the processor they actually use. diff --git a/examples/client-v2-json-processors/build.gradle.kts b/examples/client-v2-json-processors/build.gradle.kts new file mode 100644 index 000000000..e9f39b34e --- /dev/null +++ b/examples/client-v2-json-processors/build.gradle.kts @@ -0,0 +1,37 @@ +plugins { + application +} + +repositories { + mavenLocal() + mavenCentral() +} + +dependencies { + implementation(libs.clickhouseClient) + + // Keep both processors on the classpath so the example can switch between them. + implementation(libs.jacksonDatabind) + implementation(libs.gson) + + implementation(libs.slf4jApi) + runtimeOnly(libs.slf4jSimple) +} + +java { + toolchain { + languageVersion = JavaLanguageVersion.of(17) + } +} + +application { + mainClass = "com.clickhouse.examples.client_v2.json_processors.ClientV2JsonProcessorsExample" +} + +tasks.named("run") { + listOf("chEndpoint", "chUser", "chPassword", "chDatabase").forEach { key -> + System.getProperty(key)?.let { value -> + systemProperty(key, value) + } + } +} diff --git a/examples/client-v2-json-processors/gradle.properties b/examples/client-v2-json-processors/gradle.properties new file mode 100644 index 000000000..5ad69748c --- /dev/null +++ b/examples/client-v2-json-processors/gradle.properties @@ -0,0 +1 @@ +org.gradle.configuration-cache=true diff --git a/examples/client-v2-json-processors/gradle/libs.versions.toml b/examples/client-v2-json-processors/gradle/libs.versions.toml new file mode 100644 index 000000000..7379edc0e --- /dev/null +++ b/examples/client-v2-json-processors/gradle/libs.versions.toml @@ -0,0 +1,12 @@ +[versions] +clickhouseClient = "0.9.8-SNAPSHOT" +jackson = "2.18.6" +gson = "2.10.1" +slf4j = "2.0.17" + +[libraries] +clickhouseClient = { module = "com.clickhouse:client-v2", version.ref = "clickhouseClient" } +jacksonDatabind = { module = "com.fasterxml.jackson.core:jackson-databind", version.ref = "jackson" } +gson = { module = "com.google.code.gson:gson", version.ref = "gson" } +slf4jApi = { module = "org.slf4j:slf4j-api", version.ref = "slf4j" } +slf4jSimple = { module = "org.slf4j:slf4j-simple", version.ref = "slf4j" } diff --git a/examples/client-v2-json-processors/gradle/wrapper/gradle-wrapper.properties b/examples/client-v2-json-processors/gradle/wrapper/gradle-wrapper.properties new file mode 100644 index 000000000..c61a118f7 --- /dev/null +++ b/examples/client-v2-json-processors/gradle/wrapper/gradle-wrapper.properties @@ -0,0 +1,7 @@ +distributionBase=GRADLE_USER_HOME +distributionPath=wrapper/dists +distributionUrl=https\://services.gradle.org/distributions/gradle-9.4.1-bin.zip +networkTimeout=10000 +validateDistributionUrl=true +zipStoreBase=GRADLE_USER_HOME +zipStorePath=wrapper/dists diff --git a/examples/client-v2-json-processors/gradlew b/examples/client-v2-json-processors/gradlew new file mode 100755 index 000000000..739907dfd --- /dev/null +++ b/examples/client-v2-json-processors/gradlew @@ -0,0 +1,248 @@ +#!/bin/sh + +# +# Copyright © 2015 the original authors. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# https://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# SPDX-License-Identifier: Apache-2.0 +# + +############################################################################## +# +# Gradle start up script for POSIX generated by Gradle. +# +# Important for running: +# +# (1) You need a POSIX-compliant shell to run this script. If your /bin/sh is +# noncompliant, but you have some other compliant shell such as ksh or +# bash, then to run this script, type that shell name before the whole +# command line, like: +# +# ksh Gradle +# +# Busybox and similar reduced shells will NOT work, because this script +# requires all of these POSIX shell features: +# * functions; +# * expansions «$var», «${var}», «${var:-default}», «${var+SET}», +# «${var#prefix}», «${var%suffix}», and «$( cmd )»; +# * compound commands having a testable exit status, especially «case»; +# * various built-in commands including «command», «set», and «ulimit». +# +# Important for patching: +# +# (2) This script targets any POSIX shell, so it avoids extensions provided +# by Bash, Ksh, etc; in particular arrays are avoided. +# +# The "traditional" practice of packing multiple parameters into a +# space-separated string is a well documented source of bugs and security +# problems, so this is (mostly) avoided, by progressively accumulating +# options in "$@", and eventually passing that to Java. +# +# Where the inherited environment variables (DEFAULT_JVM_OPTS, JAVA_OPTS, +# and GRADLE_OPTS) rely on word-splitting, this is performed explicitly; +# see the in-line comments for details. +# +# There are tweaks for specific operating systems such as AIX, CygWin, +# Darwin, MinGW, and NonStop. +# +# (3) This script is generated from the Groovy template +# https://github.com/gradle/gradle/blob/2d6327017519d23b96af35865dc997fcb544fb40/platforms/jvm/plugins-application/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt +# within the Gradle project. +# +# You can find Gradle at https://github.com/gradle/gradle/. +# +############################################################################## + +# Attempt to set APP_HOME + +# Resolve links: $0 may be a link +app_path=$0 + +# Need this for daisy-chained symlinks. +while + APP_HOME=${app_path%"${app_path##*/}"} # leaves a trailing /; empty if no leading path + [ -h "$app_path" ] +do + ls=$( ls -ld "$app_path" ) + link=${ls#*' -> '} + case $link in #( + /*) app_path=$link ;; #( + *) app_path=$APP_HOME$link ;; + esac +done + +# This is normally unused +# shellcheck disable=SC2034 +APP_BASE_NAME=${0##*/} +# Discard cd standard output in case $CDPATH is set (https://github.com/gradle/gradle/issues/25036) +APP_HOME=$( cd -P "${APP_HOME:-./}" > /dev/null && printf '%s\n' "$PWD" ) || exit + +# Use the maximum available, or set MAX_FD != -1 to use that value. +MAX_FD=maximum + +warn () { + echo "$*" +} >&2 + +die () { + echo + echo "$*" + echo + exit 1 +} >&2 + +# OS specific support (must be 'true' or 'false'). +cygwin=false +msys=false +darwin=false +nonstop=false +case "$( uname )" in #( + CYGWIN* ) cygwin=true ;; #( + Darwin* ) darwin=true ;; #( + MSYS* | MINGW* ) msys=true ;; #( + NONSTOP* ) nonstop=true ;; +esac + + + +# Determine the Java command to use to start the JVM. +if [ -n "$JAVA_HOME" ] ; then + if [ -x "$JAVA_HOME/jre/sh/java" ] ; then + # IBM's JDK on AIX uses strange locations for the executables + JAVACMD=$JAVA_HOME/jre/sh/java + else + JAVACMD=$JAVA_HOME/bin/java + fi + if [ ! -x "$JAVACMD" ] ; then + die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME + +Please set the JAVA_HOME variable in your environment to match the +location of your Java installation." + fi +else + JAVACMD=java + if ! command -v java >/dev/null 2>&1 + then + die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. + +Please set the JAVA_HOME variable in your environment to match the +location of your Java installation." + fi +fi + +# Increase the maximum file descriptors if we can. +if ! "$cygwin" && ! "$darwin" && ! "$nonstop" ; then + case $MAX_FD in #( + max*) + # In POSIX sh, ulimit -H is undefined. That's why the result is checked to see if it worked. + # shellcheck disable=SC2039,SC3045 + MAX_FD=$( ulimit -H -n ) || + warn "Could not query maximum file descriptor limit" + esac + case $MAX_FD in #( + '' | soft) :;; #( + *) + # In POSIX sh, ulimit -n is undefined. That's why the result is checked to see if it worked. + # shellcheck disable=SC2039,SC3045 + ulimit -n "$MAX_FD" || + warn "Could not set maximum file descriptor limit to $MAX_FD" + esac +fi + +# Collect all arguments for the java command, stacking in reverse order: +# * args from the command line +# * the main class name +# * -classpath +# * -D...appname settings +# * --module-path (only if needed) +# * DEFAULT_JVM_OPTS, JAVA_OPTS, and GRADLE_OPTS environment variables. + +# For Cygwin or MSYS, switch paths to Windows format before running java +if "$cygwin" || "$msys" ; then + APP_HOME=$( cygpath --path --mixed "$APP_HOME" ) + + JAVACMD=$( cygpath --unix "$JAVACMD" ) + + # Now convert the arguments - kludge to limit ourselves to /bin/sh + for arg do + if + case $arg in #( + -*) false ;; # don't mess with options #( + /?*) t=${arg#/} t=/${t%%/*} # looks like a POSIX filepath + [ -e "$t" ] ;; #( + *) false ;; + esac + then + arg=$( cygpath --path --ignore --mixed "$arg" ) + fi + # Roll the args list around exactly as many times as the number of + # args, so each arg winds up back in the position where it started, but + # possibly modified. + # + # NB: a `for` loop captures its iteration list before it begins, so + # changing the positional parameters here affects neither the number of + # iterations, nor the values presented in `arg`. + shift # remove old arg + set -- "$@" "$arg" # push replacement arg + done +fi + + +# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"' + +# Collect all arguments for the java command: +# * DEFAULT_JVM_OPTS, JAVA_OPTS, and optsEnvironmentVar are not allowed to contain shell fragments, +# and any embedded shellness will be escaped. +# * For example: A user cannot expect ${Hostname} to be expanded, as it is an environment variable and will be +# treated as '${Hostname}' itself on the command line. + +set -- \ + "-Dorg.gradle.appname=$APP_BASE_NAME" \ + -jar "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" \ + "$@" + +# Stop when "xargs" is not available. +if ! command -v xargs >/dev/null 2>&1 +then + die "xargs is not available" +fi + +# Use "xargs" to parse quoted args. +# +# With -n1 it outputs one arg per line, with the quotes and backslashes removed. +# +# In Bash we could simply go: +# +# readarray ARGS < <( xargs -n1 <<<"$var" ) && +# set -- "${ARGS[@]}" "$@" +# +# but POSIX shell has neither arrays nor command substitution, so instead we +# post-process each arg (as a line of input to sed) to backslash-escape any +# character that might be a shell metacharacter, then use eval to reverse +# that process (while maintaining the separation between arguments), and wrap +# the whole thing up as a single "set" statement. +# +# This will of course break if any of these variables contains a newline or +# an unmatched quote. +# + +eval "set -- $( + printf '%s\n' "$DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS" | + xargs -n1 | + sed ' s~[^-[:alnum:]+,./:=@_]~\\&~g; ' | + tr '\n' ' ' + )" '"$@"' + +exec "$JAVACMD" "$@" diff --git a/examples/client-v2-json-processors/gradlew.bat b/examples/client-v2-json-processors/gradlew.bat new file mode 100644 index 000000000..e509b2dd8 --- /dev/null +++ b/examples/client-v2-json-processors/gradlew.bat @@ -0,0 +1,93 @@ +@rem +@rem Copyright 2015 the original author or authors. +@rem +@rem Licensed under the Apache License, Version 2.0 (the "License"); +@rem you may not use this file except in compliance with the License. +@rem You may obtain a copy of the License at +@rem +@rem https://www.apache.org/licenses/LICENSE-2.0 +@rem +@rem Unless required by applicable law or agreed to in writing, software +@rem distributed under the License is distributed on an "AS IS" BASIS, +@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +@rem See the License for the specific language governing permissions and +@rem limitations under the License. +@rem +@rem SPDX-License-Identifier: Apache-2.0 +@rem + +@if "%DEBUG%"=="" @echo off +@rem ########################################################################## +@rem +@rem Gradle startup script for Windows +@rem +@rem ########################################################################## + +@rem Set local scope for the variables with windows NT shell +if "%OS%"=="Windows_NT" setlocal + +set DIRNAME=%~dp0 +if "%DIRNAME%"=="" set DIRNAME=. +@rem This is normally unused +set APP_BASE_NAME=%~n0 +set APP_HOME=%DIRNAME% + +@rem Resolve any "." and ".." in APP_HOME to make it shorter. +for %%i in ("%APP_HOME%") do set APP_HOME=%%~fi + +@rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +set DEFAULT_JVM_OPTS="-Xmx64m" "-Xms64m" + +@rem Find java.exe +if defined JAVA_HOME goto findJavaFromJavaHome + +set JAVA_EXE=java.exe +%JAVA_EXE% -version >NUL 2>&1 +if %ERRORLEVEL% equ 0 goto execute + +echo. 1>&2 +echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. 1>&2 +echo. 1>&2 +echo Please set the JAVA_HOME variable in your environment to match the 1>&2 +echo location of your Java installation. 1>&2 + +goto fail + +:findJavaFromJavaHome +set JAVA_HOME=%JAVA_HOME:"=% +set JAVA_EXE=%JAVA_HOME%/bin/java.exe + +if exist "%JAVA_EXE%" goto execute + +echo. 1>&2 +echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% 1>&2 +echo. 1>&2 +echo Please set the JAVA_HOME variable in your environment to match the 1>&2 +echo location of your Java installation. 1>&2 + +goto fail + +:execute +@rem Setup the command line + + + +@rem Execute Gradle +"%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -jar "%APP_HOME%\gradle\wrapper\gradle-wrapper.jar" %* + +:end +@rem End local scope for the variables with windows NT shell +if %ERRORLEVEL% equ 0 goto mainEnd + +:fail +rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of +rem the _cmd.exe /c_ return code! +set EXIT_CODE=%ERRORLEVEL% +if %EXIT_CODE% equ 0 set EXIT_CODE=1 +if not ""=="%GRADLE_EXIT_CONSOLE%" exit %EXIT_CODE% +exit /b %EXIT_CODE% + +:mainEnd +if "%OS%"=="Windows_NT" endlocal + +:omega diff --git a/examples/client-v2-json-processors/settings.gradle.kts b/examples/client-v2-json-processors/settings.gradle.kts new file mode 100644 index 000000000..8b35c469e --- /dev/null +++ b/examples/client-v2-json-processors/settings.gradle.kts @@ -0,0 +1,5 @@ +plugins { + id("org.gradle.toolchains.foojay-resolver-convention") version "1.0.0" +} + +rootProject.name = "ch-java-client-v2-json-processors" diff --git a/examples/client-v2-json-processors/src/main/java/com/clickhouse/examples/client_v2/json_processors/ClientV2JsonProcessorsExample.java b/examples/client-v2-json-processors/src/main/java/com/clickhouse/examples/client_v2/json_processors/ClientV2JsonProcessorsExample.java new file mode 100644 index 000000000..6951bb6e2 --- /dev/null +++ b/examples/client-v2-json-processors/src/main/java/com/clickhouse/examples/client_v2/json_processors/ClientV2JsonProcessorsExample.java @@ -0,0 +1,283 @@ +package com.clickhouse.examples.client_v2.json_processors; + +import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.ClientConfigProperties; +import com.clickhouse.client.api.command.CommandResponse; +import com.clickhouse.client.api.data_formats.ClickHouseTextFormatReader; +import com.clickhouse.client.api.data_formats.GsonJsonParserFactory; +import com.clickhouse.client.api.data_formats.JSONEachRowFormatReader; +import com.clickhouse.client.api.data_formats.JacksonJsonParserFactory; +import com.clickhouse.client.api.data_formats.JsonParserFactory; +import com.clickhouse.client.api.query.QueryResponse; +import com.clickhouse.client.api.query.QuerySettings; +import com.clickhouse.data.ClickHouseFormat; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.json.JsonMapper; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.ToNumberPolicy; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Map; + +/** + * Demonstrates how to consume a {@code JSONEachRow} response with the client-v2 + * {@link JSONEachRowFormatReader} and a {@link JsonParserFactory}. + * + *

The class is intentionally written as a regular component (instance methods, + * shared {@link Client} field) so it can be copied as-is into other projects and + * have its individual methods invoked.

+ * + *

Two factories ship with the client and serve as the customization points: + * {@link com.clickhouse.client.api.data_formats.JacksonJsonParserFactory} and {@link com.clickhouse.client.api.data_formats.GsonJsonParserFactory}. Extend + * either of them and override the protected hook + * ({@code createMapper()} for Jackson, {@code customize(GsonBuilder)} for Gson) + * to plug in any library-level customization.

+ */ +public class ClientV2JsonProcessorsExample { + + private static final Logger LOG = LoggerFactory.getLogger(ClientV2JsonProcessorsExample.class); + + private static final String TABLE = "client_v2_json_processors_example"; + + /** + * Sample dataset: {@code { id, name, active, score, payload }}. + */ + private static final Object[][] SAMPLE_ROWS = { + {1, "alpha", true, 1.5, "{\"city\":\"Berlin\",\"tags\":[\"a\",\"b\"]}"}, + {2, "beta", false, 2.5, "{\"city\":\"Paris\", \"tags\":[\"c\"]}"}, + {3, "gamma", true, 3.5, "{\"city\":\"Tokyo\", \"tags\":[]}"}, + }; + + private final Client client; + + public ClientV2JsonProcessorsExample(Client client) { + this.client = client; + } + + public static void main(String[] args) throws Exception { + try (Client client = buildClient()) { + new ClientV2JsonProcessorsExample(client).run(); + } + } + + /** + * Runs the full demo: prepares the table, loads sample rows, reads them four times. + */ + public void run() throws Exception { + recreateTable(); + loadSampleData(); + + // 1. Default Jackson: use the shipped factory as-is. + readAll("Jackson (default)", new JacksonJsonParserFactory()); + + // 2. Customized Jackson: extend the factory and override createMapper(). + readAll("Jackson (custom)", new CustomJacksonParserFactory()); + + // 3. Default Gson: use the shipped factory as-is. + readAll("Gson (default)", new GsonJsonParserFactory()); + + // 4. Customized Gson: extend the factory and override customize(GsonBuilder). + readAll("Gson (custom)", new CustomGsonParserFactory()); + } + + /** + * Reads every row from {@link #TABLE} using a {@code JSONEachRow} stream + * decoded with the supplied {@link JsonParserFactory}. + * + *

When the factory also implements {@link PayloadConverter} (as the two + * custom factories below do), the raw {@code payload} value is fed through + * {@link PayloadConverter#toPayload(Object)} so the row is logged with a + * typed {@link Payload} POJO instead of the bare {@code Map} + * produced by the underlying library.

+ */ + public void readAll(String label, JsonParserFactory factory) throws Exception { + LOG.info("--- Reading rows with {} ---", label); + + QuerySettings settings = new QuerySettings() + .setFormat(ClickHouseFormat.JSONEachRow) + .setOption(ClientConfigProperties.JSON_DISABLE_NUMBER_QUOTING.getKey(), true); + String sql = "SELECT id, name, active, score, payload FROM " + TABLE + " ORDER BY id"; + + PayloadConverter converter = factory instanceof PayloadConverter ? (PayloadConverter) factory : null; + + try (QueryResponse response = client.query(sql, settings).get(); + ClickHouseTextFormatReader reader = new JSONEachRowFormatReader( + factory.createJsonParser(response.getInputStream()))) { + + while (reader.next() != null) { + Object rawPayload = reader.readValue("payload"); + Object payload = converter != null ? converter.toPayload(rawPayload) : rawPayload; + LOG.info(" id={}, name={}, active={}, score={}, payload={} ({})", + reader.getInteger("id"), + reader.getString("name"), + reader.getBoolean("active"), + reader.getDouble("score"), + payload, + payload == null ? "null" : payload.getClass().getSimpleName()); + } + } + } + + public void recreateTable() throws Exception { + execute("DROP TABLE IF EXISTS " + TABLE); + execute("CREATE TABLE " + TABLE + " (" + + "id UInt32, name String, active Bool, score Float64, payload JSON" + + ") ENGINE = MergeTree ORDER BY id"); + } + + /** + * Inserts {@link #SAMPLE_ROWS} into {@link #TABLE} as a single batched INSERT. + */ + public void loadSampleData() throws Exception { + StringBuilder sql = new StringBuilder("INSERT INTO ").append(TABLE) + .append(" (id, name, active, score, payload) VALUES"); + for (int i = 0; i < SAMPLE_ROWS.length; i++) { + Object[] row = SAMPLE_ROWS[i]; + sql.append(i == 0 ? " " : ", ") + .append('(').append(row[0]) + .append(", ").append(sqlString((String) row[1])) + .append(", ").append(row[2]) + .append(", ").append(row[3]) + .append(", ").append(sqlString((String) row[4])) + .append(')'); + } + execute(sql.toString()); + } + + private void execute(String sql) throws Exception { + try (CommandResponse ignored = client.execute(sql).get()) { + LOG.debug("Executed: {}", sql); + } + } + + private static Client buildClient() { + return new Client.Builder() + .addEndpoint(System.getProperty("chEndpoint", "http://localhost:8123")) + .setUsername(System.getProperty("chUser", "default")) + .setPassword(System.getProperty("chPassword", "")) + .setDefaultDatabase(System.getProperty("chDatabase", "default")) + .serverSetting("allow_experimental_json_type", "1") + .build(); + } + + private static String sqlString(String value) { + return "'" + value.replace("'", "''") + "'"; + } + + // --------------------------------------------------------------------- + // Customized factories + // --------------------------------------------------------------------- + + /** + * Customized {@link JacksonJsonParserFactory}. Override {@code createMapper()} + * to return any {@link ObjectMapper} you want — modules, feature flags, + * deserializers, etc. all carry over to row parsing. + * + *

This example tolerates new server-side keys and preserves big integers + * and decimals exactly inside the {@code payload} JSON column. It also + * implements {@link PayloadConverter} so the same configured mapper is + * reused to convert the row's {@code payload} {@code Map} into a typed + * {@link Payload} POJO via {@link ObjectMapper#convertValue(Object, Class)}.

+ */ + public static final class CustomJacksonParserFactory extends JacksonJsonParserFactory implements PayloadConverter { + + private static final ObjectMapper MAPPER = JsonMapper.builder() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + .configure(DeserializationFeature.USE_BIG_INTEGER_FOR_INTS, true) + .configure(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS, true) + .build(); + + @Override + protected ObjectMapper createMapper() { + return MAPPER; + } + + @Override + public Payload toPayload(Object rawPayload) { + return MAPPER.convertValue(rawPayload, Payload.class); + } + } + + /** + * Customized {@link GsonJsonParserFactory}. Override + * {@code customize(GsonBuilder)} and configure the builder; the factory + * applies {@code setLenient()} on its own afterward (which is required for + * the stream-of-objects shape of {@code JSONEachRow}). + * + *

This example parses integer-shaped JSON numbers as {@code Long} (the + * default is {@code Double}, which loses precision for large {@code Int64} + * values) and disables HTML escaping on round-trips. It also implements + * {@link PayloadConverter} so the same configured {@link Gson} is reused + * to convert the row's {@code payload} {@code Map} into a typed + * {@link Payload} POJO via {@code fromJson(toJsonTree(...))}.

+ */ + public static final class CustomGsonParserFactory extends GsonJsonParserFactory implements PayloadConverter { + + private static final Gson GSON = new GsonBuilder() + .setObjectToNumberStrategy(ToNumberPolicy.LONG_OR_DOUBLE) + .disableHtmlEscaping() + .setLenient() + .create(); + + @Override + protected void customize(GsonBuilder builder) { + builder.setObjectToNumberStrategy(ToNumberPolicy.LONG_OR_DOUBLE) + .disableHtmlEscaping(); + } + + @Override + public Payload toPayload(Object rawPayload) { + return GSON.fromJson(GSON.toJsonTree(rawPayload), Payload.class); + } + } + + // --------------------------------------------------------------------- + // Domain types used to demonstrate POJO materialization of payload + // --------------------------------------------------------------------- + + /** + * Optional hook implemented by customized factories that know how to turn + * the raw {@code payload} value (a {@code Map} produced by + * the underlying JSON library) into a typed {@link Payload} POJO. The + * default factories do not implement it, so {@link #readAll(String, JsonParserFactory)} + * logs the raw map for them. + */ + public interface PayloadConverter { + Payload toPayload(Object rawPayload); + } + + /** POJO shape of the {@code payload} JSON column used by the sample data. */ + public static final class Payload { + + private String city; + private List tags; + + public Payload() { + } + + public String getCity() { + return city; + } + + public void setCity(String city) { + this.city = city; + } + + public List getTags() { + return tags; + } + + public void setTags(List tags) { + this.tags = tags; + } + + @Override + public String toString() { + return "Payload{city='" + city + "', tags=" + tags + '}'; + } + } +} diff --git a/examples/jdbc-dispatcher-demo/.gitignore b/examples/jdbc-dispatcher-demo/.gitignore new file mode 100644 index 000000000..67bcc2f72 --- /dev/null +++ b/examples/jdbc-dispatcher-demo/.gitignore @@ -0,0 +1,2 @@ +.gradle/ +build/ diff --git a/examples/jdbc-v2-json-processors/.gitignore b/examples/jdbc-v2-json-processors/.gitignore new file mode 100644 index 000000000..f8b92c3aa --- /dev/null +++ b/examples/jdbc-v2-json-processors/.gitignore @@ -0,0 +1,2 @@ +.gradle +build diff --git a/examples/jdbc-v2-json-processors/README.md b/examples/jdbc-v2-json-processors/README.md new file mode 100644 index 000000000..80f03b24f --- /dev/null +++ b/examples/jdbc-v2-json-processors/README.md @@ -0,0 +1,193 @@ +# JDBC V2 JSON Processors Example + +## Overview + +This standalone example shows how to consume `FORMAT JSONEachRow` responses +through `jdbc-v2` with the two factories shipped under +`com.clickhouse.client.api.data_formats`: + +- `JacksonJsonParserFactory` +- `GsonJsonParserFactory` + +### How the JDBC driver selects a factory + +The driver picks the parser factory **by fully-qualified class name** from +the `jdbc_json_parser_factory` driver property. +The value is the FQN of a class that implements `JsonParserFactory`; the +driver loads it reflectively and instantiates it through a **public no-arg +constructor**. There is no enum-style selector. + +Selection is **connection-level**: the factory cannot be swapped on an +existing connection. The driver instantiates the named class once during +connection creation and reuses that instance for every `JSONEachRow` +response served by the connection. + +### Customization is done by extending the factory + +Because the driver instantiates the named class with a no-arg constructor, +customization cannot be expressed as constructor arguments. The supported +approach is: + +1. Subclass `JacksonJsonParserFactory` or `GsonJsonParserFactory` in your + own code. +2. Override the protected hook: + - `protected ObjectMapper createMapper()` on `JacksonJsonParserFactory` — + return any fully-configured `ObjectMapper` (modules, feature flags, + deserializers). + - `protected void customize(GsonBuilder builder)` on + `GsonJsonParserFactory` — configure the `GsonBuilder` (number policy, + `TypeAdapter`s, date format, ...). The factory still applies + `setLenient()` on its own afterwards, which is required for + `JSONEachRow`. +3. Set `JSON_PARSER_FACTORY` to the FQN of the subclass. + +This example carries both subclasses as `public static final` nested classes +inside `JdbcV2JsonProcessorsExample` (`CustomJacksonParserFactory` and +`CustomGsonParserFactory`). The example feeds their FQNs to the driver via +`factoryClass.getName()`, which for nested classes returns the +`Outer$Inner` binary form — accepted by `Class.forName(...)` and by the +driver. If you set `JSON_PARSER_FACTORY` manually (e.g. from a config file +or JDBC URL) and your subclass is nested, you must use the same `$`-form; +top-level classes use the ordinary dot-separated FQN. + +Both custom subclasses also implement a tiny `PayloadConverter` interface +defined inside the example: their configured `ObjectMapper` / `Gson` is +reused to convert the raw `payload` value produced by the underlying +library into a typed `Payload` POJO. Because the JDBC driver only exposes +the factory through the connection (not as a Java object), `readAll(...)` +detects the interface on the factory **class** and instantiates its own +converter via the same public no-arg constructor the driver uses. The +default factories do not implement the interface, so `readAll(...)` logs +the raw map for them — making the contrast between the default behavior +and the customized behavior visible in the output. + +### Component shape + +`JdbcV2JsonProcessorsExample` is written as a small component: + +- `JdbcV2JsonProcessorsExample(String url, String user, String password)` + holds the connection settings and exposes regular instance methods + (`recreateTable()`, `loadSampleData()`, `readAll(label, factoryClass)`, + `run()`), so the class can be copied as-is into another project and have + its individual methods invoked. +- Sample rows are kept in a plain `Object[][]` constant, separate from the + SQL, so the read path stays focused on the parser-factory wiring. + +Each read call in `run()` follows the same three-step shape: + +1. **Pick a factory class** — `JacksonJsonParserFactory.class` / + `GsonJsonParserFactory.class` for defaults, or one of the nested custom + subclasses. +2. **Customize if needed** — only inside the subclass, by overriding the + protected hook. +3. **Execute** — `readAll(label, factoryClass)` opens a fresh connection + with `JSON_PARSER_FACTORY=`, runs the `SELECT ... FORMAT JSONEachRow` + and iterates the `ResultSet`. + +Because JDBC selects `JSONEachRow` through SQL text, set the JSON output +server settings explicitly on the connection when numeric accessors are used: + +```java +props.setProperty(ClientConfigProperties.serverSetting("output_format_json_quote_64bit_integers"), "0"); +props.setProperty(ClientConfigProperties.serverSetting("output_format_json_quote_64bit_floats"), "0"); +props.setProperty(ClientConfigProperties.serverSetting("output_format_json_quote_decimals"), "0"); +``` + +Denormal floating-point values (`NaN`, `Inf`, `-Inf`) are not handled by the +built-in JSON reader yet. Keep `output_format_json_quote_denormals=1` and +handle those values as strings if your queries can return them. + +## Integer Precision + +ClickHouse 64-bit integers can be larger than the exact integer range of a +JSON floating-point number. Jackson's default map materialization preserves +ordinary integer tokens as integer `Number` values. Gson's default +`Map` materialization may surface numbers as floating-point +values, which can round large integers before `ResultSet.getLong(...)` sees +them. + +For Gson, extend `GsonJsonParserFactory` and configure the object number +strategy: + +```java +public final class PreciseGsonJsonParserFactory extends GsonJsonParserFactory { + @Override + protected void customize(GsonBuilder builder) { + builder.setObjectToNumberStrategy(com.google.gson.ToNumberPolicy.LONG_OR_DOUBLE); + } +} +``` + +Then configure JDBC with the factory class name: + +```java +props.setProperty(DriverProperties.JSON_PARSER_FACTORY.getKey(), + PreciseGsonJsonParserFactory.class.getName()); +``` + +The included `CustomGsonParserFactory` uses this pattern. Use +`ToNumberPolicy.BIG_DECIMAL` instead when exact decimal representation matters +more than receiving integer tokens as `Long`. + +## Requirements + +- JDK 17 or newer +- A running ClickHouse server reachable from the machine running the example +- A locally installed `jdbc-v2` snapshot from this repository + +## How to Run + +From this directory: + +```shell +gradle run +``` + +Connection properties can be supplied as system properties: + +- `-DchUrl` — JDBC URL (default: `jdbc:clickhouse://localhost:8123/default`) +- `-DchUser` — ClickHouse user name (default: `default`) +- `-DchPassword` — ClickHouse user password (default: empty) + +Example with custom connection properties: + +```shell +gradle run \ + -DchUrl=jdbc:clickhouse://localhost:8123/default \ + -DchUser=default \ + -DchPassword= +``` + +## Executable Example + +`com.clickhouse.examples.jdbc_v2.json_processors.JdbcV2JsonProcessorsExample` + +Steps performed by `run()`: + +1. `recreateTable()` — drops and re-creates `jdbc_v2_json_processors_example` + with primitive columns and one `payload JSON` column. +2. `loadSampleData()` — inserts the rows from the `SAMPLE_ROWS` array as a + single batched `INSERT`. +3. `readAll(...)` is invoked four times, each time pointing + `JSON_PARSER_FACTORY` at a different class: + - `JacksonJsonParserFactory` — default Jackson; + - `JdbcV2JsonProcessorsExample.CustomJacksonParserFactory` — nested + subclass overriding `createMapper()` to tolerate unknown properties + and preserve big integers and decimals exactly, also implementing + `PayloadConverter` to convert each row's `payload` `Map` into a + `Payload` POJO via `ObjectMapper.convertValue(...)`; + - `GsonJsonParserFactory` — default Gson; + - `JdbcV2JsonProcessorsExample.CustomGsonParserFactory` — nested + subclass overriding `customize(GsonBuilder)` to use a `LONG_OR_DOUBLE` + number policy and disable HTML escaping, also implementing + `PayloadConverter` to convert each row's `payload` `Map` into a + `Payload` POJO via `gson.fromJson(gson.toJsonTree(...))`. + +Logged rows include the payload value's runtime class so the difference +between the default factories (which surface a `LinkedHashMap` / +`LinkedTreeMap`) and the customized factories (which surface a `Payload`) +shows up directly in the output. + +The build keeps both `jackson-databind` and `gson` on the classpath so the +example can switch between processors at runtime. Production applications +only need to keep the processor they actually use. diff --git a/examples/jdbc-v2-json-processors/build.gradle.kts b/examples/jdbc-v2-json-processors/build.gradle.kts new file mode 100644 index 000000000..07ed830d2 --- /dev/null +++ b/examples/jdbc-v2-json-processors/build.gradle.kts @@ -0,0 +1,37 @@ +plugins { + application +} + +repositories { + mavenLocal() + mavenCentral() +} + +dependencies { + implementation(libs.jdbcV2) + + // Keep both processors on the classpath so the example can switch between them. + implementation(libs.jacksonDatabind) + implementation(libs.gson) + + implementation(libs.slf4jApi) + runtimeOnly(libs.slf4jSimple) +} + +java { + toolchain { + languageVersion = JavaLanguageVersion.of(17) + } +} + +application { + mainClass = "com.clickhouse.examples.jdbc_v2.json_processors.JdbcV2JsonProcessorsExample" +} + +tasks.named("run") { + listOf("chUrl", "chUser", "chPassword").forEach { key -> + System.getProperty(key)?.let { value -> + systemProperty(key, value) + } + } +} diff --git a/examples/jdbc-v2-json-processors/gradle.properties b/examples/jdbc-v2-json-processors/gradle.properties new file mode 100644 index 000000000..5ad69748c --- /dev/null +++ b/examples/jdbc-v2-json-processors/gradle.properties @@ -0,0 +1 @@ +org.gradle.configuration-cache=true diff --git a/examples/jdbc-v2-json-processors/gradle/libs.versions.toml b/examples/jdbc-v2-json-processors/gradle/libs.versions.toml new file mode 100644 index 000000000..6ac8a57a6 --- /dev/null +++ b/examples/jdbc-v2-json-processors/gradle/libs.versions.toml @@ -0,0 +1,12 @@ +[versions] +jdbcV2 = "0.9.8-SNAPSHOT" +jackson = "2.18.6" +gson = "2.10.1" +slf4j = "2.0.17" + +[libraries] +jdbcV2 = { module = "com.clickhouse:jdbc-v2", version.ref = "jdbcV2" } +jacksonDatabind = { module = "com.fasterxml.jackson.core:jackson-databind", version.ref = "jackson" } +gson = { module = "com.google.code.gson:gson", version.ref = "gson" } +slf4jApi = { module = "org.slf4j:slf4j-api", version.ref = "slf4j" } +slf4jSimple = { module = "org.slf4j:slf4j-simple", version.ref = "slf4j" } diff --git a/examples/jdbc-v2-json-processors/gradle/wrapper/gradle-wrapper.properties b/examples/jdbc-v2-json-processors/gradle/wrapper/gradle-wrapper.properties new file mode 100644 index 000000000..c61a118f7 --- /dev/null +++ b/examples/jdbc-v2-json-processors/gradle/wrapper/gradle-wrapper.properties @@ -0,0 +1,7 @@ +distributionBase=GRADLE_USER_HOME +distributionPath=wrapper/dists +distributionUrl=https\://services.gradle.org/distributions/gradle-9.4.1-bin.zip +networkTimeout=10000 +validateDistributionUrl=true +zipStoreBase=GRADLE_USER_HOME +zipStorePath=wrapper/dists diff --git a/examples/jdbc-v2-json-processors/gradlew b/examples/jdbc-v2-json-processors/gradlew new file mode 100755 index 000000000..739907dfd --- /dev/null +++ b/examples/jdbc-v2-json-processors/gradlew @@ -0,0 +1,248 @@ +#!/bin/sh + +# +# Copyright © 2015 the original authors. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# https://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# SPDX-License-Identifier: Apache-2.0 +# + +############################################################################## +# +# Gradle start up script for POSIX generated by Gradle. +# +# Important for running: +# +# (1) You need a POSIX-compliant shell to run this script. If your /bin/sh is +# noncompliant, but you have some other compliant shell such as ksh or +# bash, then to run this script, type that shell name before the whole +# command line, like: +# +# ksh Gradle +# +# Busybox and similar reduced shells will NOT work, because this script +# requires all of these POSIX shell features: +# * functions; +# * expansions «$var», «${var}», «${var:-default}», «${var+SET}», +# «${var#prefix}», «${var%suffix}», and «$( cmd )»; +# * compound commands having a testable exit status, especially «case»; +# * various built-in commands including «command», «set», and «ulimit». +# +# Important for patching: +# +# (2) This script targets any POSIX shell, so it avoids extensions provided +# by Bash, Ksh, etc; in particular arrays are avoided. +# +# The "traditional" practice of packing multiple parameters into a +# space-separated string is a well documented source of bugs and security +# problems, so this is (mostly) avoided, by progressively accumulating +# options in "$@", and eventually passing that to Java. +# +# Where the inherited environment variables (DEFAULT_JVM_OPTS, JAVA_OPTS, +# and GRADLE_OPTS) rely on word-splitting, this is performed explicitly; +# see the in-line comments for details. +# +# There are tweaks for specific operating systems such as AIX, CygWin, +# Darwin, MinGW, and NonStop. +# +# (3) This script is generated from the Groovy template +# https://github.com/gradle/gradle/blob/2d6327017519d23b96af35865dc997fcb544fb40/platforms/jvm/plugins-application/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt +# within the Gradle project. +# +# You can find Gradle at https://github.com/gradle/gradle/. +# +############################################################################## + +# Attempt to set APP_HOME + +# Resolve links: $0 may be a link +app_path=$0 + +# Need this for daisy-chained symlinks. +while + APP_HOME=${app_path%"${app_path##*/}"} # leaves a trailing /; empty if no leading path + [ -h "$app_path" ] +do + ls=$( ls -ld "$app_path" ) + link=${ls#*' -> '} + case $link in #( + /*) app_path=$link ;; #( + *) app_path=$APP_HOME$link ;; + esac +done + +# This is normally unused +# shellcheck disable=SC2034 +APP_BASE_NAME=${0##*/} +# Discard cd standard output in case $CDPATH is set (https://github.com/gradle/gradle/issues/25036) +APP_HOME=$( cd -P "${APP_HOME:-./}" > /dev/null && printf '%s\n' "$PWD" ) || exit + +# Use the maximum available, or set MAX_FD != -1 to use that value. +MAX_FD=maximum + +warn () { + echo "$*" +} >&2 + +die () { + echo + echo "$*" + echo + exit 1 +} >&2 + +# OS specific support (must be 'true' or 'false'). +cygwin=false +msys=false +darwin=false +nonstop=false +case "$( uname )" in #( + CYGWIN* ) cygwin=true ;; #( + Darwin* ) darwin=true ;; #( + MSYS* | MINGW* ) msys=true ;; #( + NONSTOP* ) nonstop=true ;; +esac + + + +# Determine the Java command to use to start the JVM. +if [ -n "$JAVA_HOME" ] ; then + if [ -x "$JAVA_HOME/jre/sh/java" ] ; then + # IBM's JDK on AIX uses strange locations for the executables + JAVACMD=$JAVA_HOME/jre/sh/java + else + JAVACMD=$JAVA_HOME/bin/java + fi + if [ ! -x "$JAVACMD" ] ; then + die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME + +Please set the JAVA_HOME variable in your environment to match the +location of your Java installation." + fi +else + JAVACMD=java + if ! command -v java >/dev/null 2>&1 + then + die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. + +Please set the JAVA_HOME variable in your environment to match the +location of your Java installation." + fi +fi + +# Increase the maximum file descriptors if we can. +if ! "$cygwin" && ! "$darwin" && ! "$nonstop" ; then + case $MAX_FD in #( + max*) + # In POSIX sh, ulimit -H is undefined. That's why the result is checked to see if it worked. + # shellcheck disable=SC2039,SC3045 + MAX_FD=$( ulimit -H -n ) || + warn "Could not query maximum file descriptor limit" + esac + case $MAX_FD in #( + '' | soft) :;; #( + *) + # In POSIX sh, ulimit -n is undefined. That's why the result is checked to see if it worked. + # shellcheck disable=SC2039,SC3045 + ulimit -n "$MAX_FD" || + warn "Could not set maximum file descriptor limit to $MAX_FD" + esac +fi + +# Collect all arguments for the java command, stacking in reverse order: +# * args from the command line +# * the main class name +# * -classpath +# * -D...appname settings +# * --module-path (only if needed) +# * DEFAULT_JVM_OPTS, JAVA_OPTS, and GRADLE_OPTS environment variables. + +# For Cygwin or MSYS, switch paths to Windows format before running java +if "$cygwin" || "$msys" ; then + APP_HOME=$( cygpath --path --mixed "$APP_HOME" ) + + JAVACMD=$( cygpath --unix "$JAVACMD" ) + + # Now convert the arguments - kludge to limit ourselves to /bin/sh + for arg do + if + case $arg in #( + -*) false ;; # don't mess with options #( + /?*) t=${arg#/} t=/${t%%/*} # looks like a POSIX filepath + [ -e "$t" ] ;; #( + *) false ;; + esac + then + arg=$( cygpath --path --ignore --mixed "$arg" ) + fi + # Roll the args list around exactly as many times as the number of + # args, so each arg winds up back in the position where it started, but + # possibly modified. + # + # NB: a `for` loop captures its iteration list before it begins, so + # changing the positional parameters here affects neither the number of + # iterations, nor the values presented in `arg`. + shift # remove old arg + set -- "$@" "$arg" # push replacement arg + done +fi + + +# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"' + +# Collect all arguments for the java command: +# * DEFAULT_JVM_OPTS, JAVA_OPTS, and optsEnvironmentVar are not allowed to contain shell fragments, +# and any embedded shellness will be escaped. +# * For example: A user cannot expect ${Hostname} to be expanded, as it is an environment variable and will be +# treated as '${Hostname}' itself on the command line. + +set -- \ + "-Dorg.gradle.appname=$APP_BASE_NAME" \ + -jar "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" \ + "$@" + +# Stop when "xargs" is not available. +if ! command -v xargs >/dev/null 2>&1 +then + die "xargs is not available" +fi + +# Use "xargs" to parse quoted args. +# +# With -n1 it outputs one arg per line, with the quotes and backslashes removed. +# +# In Bash we could simply go: +# +# readarray ARGS < <( xargs -n1 <<<"$var" ) && +# set -- "${ARGS[@]}" "$@" +# +# but POSIX shell has neither arrays nor command substitution, so instead we +# post-process each arg (as a line of input to sed) to backslash-escape any +# character that might be a shell metacharacter, then use eval to reverse +# that process (while maintaining the separation between arguments), and wrap +# the whole thing up as a single "set" statement. +# +# This will of course break if any of these variables contains a newline or +# an unmatched quote. +# + +eval "set -- $( + printf '%s\n' "$DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS" | + xargs -n1 | + sed ' s~[^-[:alnum:]+,./:=@_]~\\&~g; ' | + tr '\n' ' ' + )" '"$@"' + +exec "$JAVACMD" "$@" diff --git a/examples/jdbc-v2-json-processors/gradlew.bat b/examples/jdbc-v2-json-processors/gradlew.bat new file mode 100644 index 000000000..e509b2dd8 --- /dev/null +++ b/examples/jdbc-v2-json-processors/gradlew.bat @@ -0,0 +1,93 @@ +@rem +@rem Copyright 2015 the original author or authors. +@rem +@rem Licensed under the Apache License, Version 2.0 (the "License"); +@rem you may not use this file except in compliance with the License. +@rem You may obtain a copy of the License at +@rem +@rem https://www.apache.org/licenses/LICENSE-2.0 +@rem +@rem Unless required by applicable law or agreed to in writing, software +@rem distributed under the License is distributed on an "AS IS" BASIS, +@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +@rem See the License for the specific language governing permissions and +@rem limitations under the License. +@rem +@rem SPDX-License-Identifier: Apache-2.0 +@rem + +@if "%DEBUG%"=="" @echo off +@rem ########################################################################## +@rem +@rem Gradle startup script for Windows +@rem +@rem ########################################################################## + +@rem Set local scope for the variables with windows NT shell +if "%OS%"=="Windows_NT" setlocal + +set DIRNAME=%~dp0 +if "%DIRNAME%"=="" set DIRNAME=. +@rem This is normally unused +set APP_BASE_NAME=%~n0 +set APP_HOME=%DIRNAME% + +@rem Resolve any "." and ".." in APP_HOME to make it shorter. +for %%i in ("%APP_HOME%") do set APP_HOME=%%~fi + +@rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +set DEFAULT_JVM_OPTS="-Xmx64m" "-Xms64m" + +@rem Find java.exe +if defined JAVA_HOME goto findJavaFromJavaHome + +set JAVA_EXE=java.exe +%JAVA_EXE% -version >NUL 2>&1 +if %ERRORLEVEL% equ 0 goto execute + +echo. 1>&2 +echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. 1>&2 +echo. 1>&2 +echo Please set the JAVA_HOME variable in your environment to match the 1>&2 +echo location of your Java installation. 1>&2 + +goto fail + +:findJavaFromJavaHome +set JAVA_HOME=%JAVA_HOME:"=% +set JAVA_EXE=%JAVA_HOME%/bin/java.exe + +if exist "%JAVA_EXE%" goto execute + +echo. 1>&2 +echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% 1>&2 +echo. 1>&2 +echo Please set the JAVA_HOME variable in your environment to match the 1>&2 +echo location of your Java installation. 1>&2 + +goto fail + +:execute +@rem Setup the command line + + + +@rem Execute Gradle +"%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -jar "%APP_HOME%\gradle\wrapper\gradle-wrapper.jar" %* + +:end +@rem End local scope for the variables with windows NT shell +if %ERRORLEVEL% equ 0 goto mainEnd + +:fail +rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of +rem the _cmd.exe /c_ return code! +set EXIT_CODE=%ERRORLEVEL% +if %EXIT_CODE% equ 0 set EXIT_CODE=1 +if not ""=="%GRADLE_EXIT_CONSOLE%" exit %EXIT_CODE% +exit /b %EXIT_CODE% + +:mainEnd +if "%OS%"=="Windows_NT" endlocal + +:omega diff --git a/examples/jdbc-v2-json-processors/settings.gradle.kts b/examples/jdbc-v2-json-processors/settings.gradle.kts new file mode 100644 index 000000000..4cb564e86 --- /dev/null +++ b/examples/jdbc-v2-json-processors/settings.gradle.kts @@ -0,0 +1,5 @@ +plugins { + id("org.gradle.toolchains.foojay-resolver-convention") version "1.0.0" +} + +rootProject.name = "ch-java-jdbc-v2-json-processors" diff --git a/examples/jdbc-v2-json-processors/src/main/java/com/clickhouse/examples/jdbc_v2/json_processors/JdbcV2JsonProcessorsExample.java b/examples/jdbc-v2-json-processors/src/main/java/com/clickhouse/examples/jdbc_v2/json_processors/JdbcV2JsonProcessorsExample.java new file mode 100644 index 000000000..bccb90c92 --- /dev/null +++ b/examples/jdbc-v2-json-processors/src/main/java/com/clickhouse/examples/jdbc_v2/json_processors/JdbcV2JsonProcessorsExample.java @@ -0,0 +1,329 @@ +package com.clickhouse.examples.jdbc_v2.json_processors; + +import com.clickhouse.client.api.ClientConfigProperties; +import com.clickhouse.client.api.data_formats.GsonJsonParserFactory; +import com.clickhouse.client.api.data_formats.JacksonJsonParserFactory; +import com.clickhouse.client.api.data_formats.JsonParserFactory; +import com.clickhouse.jdbc.Driver; +import com.clickhouse.jdbc.DriverProperties; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.json.JsonMapper; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.ToNumberPolicy; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.List; +import java.util.Properties; + +/** + * Demonstrates how to consume {@code FORMAT JSONEachRow} responses through + * {@code jdbc-v2} with the two factories shipped under + * {@link com.clickhouse.client.api.data_formats}. + * + *

The class is intentionally written as a regular component (instance + * methods, shared connection settings as fields) so it can be copied as-is + * into other projects and have its individual methods invoked.

+ * + *

The JDBC driver selects a parser factory from the + * {@link DriverProperties#JSON_PARSER_FACTORY} driver property — its value is + * the fully-qualified class name of a {@link JsonParserFactory} + * implementation, which the driver loads reflectively and instantiates via + * a public no-arg constructor. There is no enum-style selector.

+ * + *

For Jackson/Gson customization the recommended approach is therefore to + * extend the shipped factory and override its protected hook + * ({@code createMapper()} or {@code customize(GsonBuilder)}), then point + * {@code JSON_PARSER_FACTORY} at the FQN of the subclass. See + * {@link CustomJacksonParserFactory} and {@link CustomGsonParserFactory} + * below.

+ */ +public class JdbcV2JsonProcessorsExample { + + private static final Logger LOG = LoggerFactory.getLogger(JdbcV2JsonProcessorsExample.class); + + private static final String TABLE = "jdbc_v2_json_processors_example"; + + /** + * Sample dataset: {@code { id, name, active, score, payload }}. + */ + private static final Object[][] SAMPLE_ROWS = { + {1, "alpha", true, 1.5, "{\"city\":\"Berlin\",\"tags\":[\"a\",\"b\"]}"}, + {2, "beta", false, 2.5, "{\"city\":\"Paris\", \"tags\":[\"c\"]}"}, + {3, "gamma", true, 3.5, "{\"city\":\"Tokyo\", \"tags\":[]}"}, + }; + + private final String url; + private final String user; + private final String password; + + public JdbcV2JsonProcessorsExample(String url, String user, String password) { + this.url = url; + this.user = user; + this.password = password; + } + + public static void main(String[] args) throws Exception { + // jdbc-v2 does not self-register from a static initializer; standalone + // examples must register the driver explicitly before calling DriverManager. + Driver.load(); + + String url = System.getProperty("chUrl", "jdbc:clickhouse://localhost:8123/default"); + String user = System.getProperty("chUser", "default"); + String password = System.getProperty("chPassword", ""); + new JdbcV2JsonProcessorsExample(url, user, password).run(); + } + + /** + * Runs the full demo: prepares the table, loads sample rows, reads them four times. + */ + public void run() throws Exception { + recreateTable(); + loadSampleData(); + + // 1. Default Jackson: factory FQN points to the shipped class. + readAll("Jackson (default)", JacksonJsonParserFactory.class); + + // 2. Customized Jackson: factory FQN points to a subclass of + // JacksonJsonParserFactory whose no-arg constructor overrides createMapper(). + readAll("Jackson (custom)", CustomJacksonParserFactory.class); + + // 3. Default Gson. + readAll("Gson (default)", GsonJsonParserFactory.class); + + // 4. Customized Gson: subclass overriding customize(GsonBuilder). + readAll("Gson (custom)", CustomGsonParserFactory.class); + } + + /** + * Reads every row from {@link #TABLE} through a fresh JDBC connection + * configured to use the supplied {@link JsonParserFactory} implementation. + * + *

Selection is connection-level: the factory cannot be swapped on an + * existing connection. The driver instantiates the named class once during + * connection creation and reuses that instance for every {@code JSONEachRow} + * response served by the connection.

+ * + *

When {@code factoryClass} also implements {@link PayloadConverter} + * (as the two custom factories below do), the method also instantiates a + * converter — through the same public no-arg constructor the driver uses — + * and feeds {@code rs.getObject("payload")} through + * {@link PayloadConverter#toPayload(Object)}, so the row is logged with a + * typed {@link Payload} POJO instead of the bare {@code Map} + * produced by the underlying library.

+ */ + public void readAll(String label, Class factoryClass) throws Exception { + LOG.info("--- Reading rows with {} ({}) ---", label, factoryClass.getName()); + + Properties props = baseProperties(); + props.setProperty(DriverProperties.JSON_PARSER_FACTORY.getKey(), factoryClass.getName()); + + PayloadConverter converter = PayloadConverter.class.isAssignableFrom(factoryClass) + ? (PayloadConverter) factoryClass.getDeclaredConstructor().newInstance() + : null; + + String sql = "SELECT id, name, active, score, payload FROM " + TABLE + + " ORDER BY id FORMAT JSONEachRow"; + + try (Connection connection = DriverManager.getConnection(url, props); + Statement statement = connection.createStatement(); + ResultSet rs = statement.executeQuery(sql)) { + + while (rs.next()) { + Object rawPayload = rs.getObject("payload"); + Object payload = converter != null ? converter.toPayload(rawPayload) : rawPayload; + LOG.info(" id={}, name={}, active={}, score={}, payload={} ({})", + rs.getInt("id"), + rs.getString("name"), + rs.getBoolean("active"), + rs.getDouble("score"), + payload, + payload == null ? "null" : payload.getClass().getSimpleName()); + } + } + } + + public void recreateTable() throws SQLException { + try (Connection connection = DriverManager.getConnection(url, baseProperties()); + Statement statement = connection.createStatement()) { + statement.execute("DROP TABLE IF EXISTS " + TABLE); + statement.execute("CREATE TABLE " + TABLE + " (" + + "id UInt32, name String, active Bool, score Float64, payload JSON" + + ") ENGINE = MergeTree ORDER BY id"); + } + } + + /** + * Inserts {@link #SAMPLE_ROWS} into {@link #TABLE} as a single batched INSERT. + */ + public void loadSampleData() throws SQLException { + StringBuilder sql = new StringBuilder("INSERT INTO ").append(TABLE) + .append(" (id, name, active, score, payload) VALUES"); + for (int i = 0; i < SAMPLE_ROWS.length; i++) { + Object[] row = SAMPLE_ROWS[i]; + sql.append(i == 0 ? " " : ", ") + .append('(').append(row[0]) + .append(", ").append(sqlString((String) row[1])) + .append(", ").append(row[2]) + .append(", ").append(row[3]) + .append(", ").append(sqlString((String) row[4])) + .append(')'); + } + try (Connection connection = DriverManager.getConnection(url, baseProperties()); + Statement statement = connection.createStatement()) { + statement.executeUpdate(sql.toString()); + } + } + + private Properties baseProperties() { + Properties properties = new Properties(); + properties.setProperty("user", user); + properties.setProperty("password", password); + properties.setProperty(ClientConfigProperties.serverSetting("allow_experimental_json_type"), "1"); + properties.setProperty(ClientConfigProperties.serverSetting("output_format_json_quote_64bit_integers"), "0"); + properties.setProperty(ClientConfigProperties.serverSetting("output_format_json_quote_64bit_floats"), "0"); + properties.setProperty(ClientConfigProperties.serverSetting("output_format_json_quote_decimals"), "0"); + return properties; + } + + private static String sqlString(String value) { + return "'" + value.replace("'", "''") + "'"; + } + + // --------------------------------------------------------------------- + // Customized factories + // --------------------------------------------------------------------- + + /** + * Customized {@link JacksonJsonParserFactory}. Override {@code createMapper()} + * to return any {@link ObjectMapper} you want — modules, feature flags, + * deserializers, etc. all carry over to row parsing. + * + *

This class must be {@code public static} with a public no-arg + * constructor because the JDBC driver loads it reflectively via the + * {@code jdbc_json_parser_factory} driver property; the {@code .getName()} + * of a nested class is the {@code Outer$Inner} binary form, which + * {@code Class.forName(...)} accepts.

+ * + *

This example tolerates new server-side keys and preserves big integers + * and decimals exactly inside the {@code payload} JSON column. It also + * implements {@link PayloadConverter} so the same configured mapper is + * reused to convert the row's {@code payload} {@code Map} into a typed + * {@link Payload} POJO via {@link ObjectMapper#convertValue(Object, Class)}.

+ */ + public static final class CustomJacksonParserFactory extends JacksonJsonParserFactory implements PayloadConverter { + + private static final ObjectMapper MAPPER = JsonMapper.builder() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + .configure(DeserializationFeature.USE_BIG_INTEGER_FOR_INTS, true) + .configure(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS, true) + .build(); + + @Override + protected ObjectMapper createMapper() { + return MAPPER; + } + + @Override + public Payload toPayload(Object rawPayload) { + return MAPPER.convertValue(rawPayload, Payload.class); + } + } + + /** + * Customized {@link GsonJsonParserFactory}. Override + * {@code customize(GsonBuilder)} and configure the builder; the factory + * applies {@code setLenient()} on its own afterwards (which is required for + * the stream-of-objects shape of {@code JSONEachRow}). + * + *

This class must be {@code public static} with a public no-arg + * constructor because the JDBC driver loads it reflectively via the + * {@code jdbc_json_parser_factory} driver property; the {@code .getName()} + * of a nested class is the {@code Outer$Inner} binary form, which + * {@code Class.forName(...)} accepts.

+ * + *

This example parses integer-shaped JSON numbers as {@code Long} (the + * default is {@code Double}, which loses precision for large {@code Int64} + * values) and disables HTML escaping on round-trips. It also implements + * {@link PayloadConverter} so the same configured {@link Gson} is reused + * to convert the row's {@code payload} {@code Map} into a typed + * {@link Payload} POJO via {@code fromJson(toJsonTree(...))}.

+ */ + public static final class CustomGsonParserFactory extends GsonJsonParserFactory implements PayloadConverter { + + private static final Gson GSON = new GsonBuilder() + .setObjectToNumberStrategy(ToNumberPolicy.LONG_OR_DOUBLE) + .disableHtmlEscaping() + .setLenient() + .create(); + + @Override + protected void customize(GsonBuilder builder) { + builder.setObjectToNumberStrategy(ToNumberPolicy.LONG_OR_DOUBLE) + .disableHtmlEscaping(); + } + + @Override + public Payload toPayload(Object rawPayload) { + return GSON.fromJson(GSON.toJsonTree(rawPayload), Payload.class); + } + } + + // --------------------------------------------------------------------- + // Domain types used to demonstrate POJO materialization of payload + // --------------------------------------------------------------------- + + /** + * Optional hook implemented by customized factories that know how to turn + * the raw {@code payload} value (a {@code Map} produced by + * the underlying JSON library) into a typed {@link Payload} POJO. The + * default factories do not implement it, so + * {@link #readAll(String, Class)} logs the raw map for them. + * + *

The interface is invoked from application code, not from the JDBC + * driver itself: {@code readAll(...)} detects it on the factory class and + * instantiates its own converter through the same public no-arg constructor + * the driver uses for row parsing.

+ */ + public interface PayloadConverter { + Payload toPayload(Object rawPayload); + } + + /** POJO shape of the {@code payload} JSON column used by the sample data. */ + public static final class Payload { + + private String city; + private List tags; + + public Payload() { + } + + public String getCity() { + return city; + } + + public void setCity(String city) { + this.city = city; + } + + public List getTags() { + return tags; + } + + public void setTags(List tags) { + this.tags = tags; + } + + @Override + public String toString() { + return "Payload{city='" + city + "', tags=" + tags + '}'; + } + } +} diff --git a/jdbc-v2/pom.xml b/jdbc-v2/pom.xml index a8f02f2e9..05b035694 100644 --- a/jdbc-v2/pom.xml +++ b/jdbc-v2/pom.xml @@ -50,14 +50,35 @@ ${guava.version} - com.fasterxml.jackson.core jackson-databind - test ${jackson.version} + provided + + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + provided + + com.fasterxml.jackson.core + jackson-annotations + ${jackson.version} + provided + + + + com.google.code.gson + gson + ${gson.version} + provided + + + ${project.parent.groupId} clickhouse-client @@ -89,18 +110,6 @@ test - - com.fasterxml.jackson.core - jackson-core - ${jackson.version} - test - - - com.fasterxml.jackson.core - jackson-databind - ${jackson.version} - test - com.fasterxml.jackson.dataformat jackson-dataformat-yaml @@ -211,4 +220,4 @@ - \ No newline at end of file + diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/ConnectionImpl.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/ConnectionImpl.java index 261acc5b7..82b3ff153 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/ConnectionImpl.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/ConnectionImpl.java @@ -2,6 +2,7 @@ import com.clickhouse.client.api.Client; import com.clickhouse.client.api.ClientConfigProperties; +import com.clickhouse.client.api.data_formats.JsonParserFactory; import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.GenericRecord; import com.clickhouse.client.api.query.QuerySettings; @@ -17,6 +18,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.lang.reflect.InvocationTargetException; import java.sql.Array; import java.sql.Blob; import java.sql.CallableStatement; @@ -70,6 +72,8 @@ public class ConnectionImpl implements Connection, JdbcV2Wrapper { private final FeatureManager featureManager; private volatile ImmutableMap> typeMap; + private final JsonParserFactory jsonParserFactory; + public ConnectionImpl(String url, Properties info) throws SQLException { try { this.url = url;//Raw URL @@ -121,6 +125,10 @@ public ConnectionImpl(String url, Properties info) throws SQLException { DriverProperties.SQL_PARSER.getDefaultValue()), config); this.featureManager = new FeatureManager(this.config); this.typeMap = ImmutableMap.>builder().putAll(this.config.getTypeMap()).buildKeepingLast(); + + final String jsonParserFactoryName = config.getDriverProperty(DriverProperties.JSON_PARSER_FACTORY.getKey(), null); + this.jsonParserFactory = jsonParserFactoryName == null ? null : instantiateJsonParserFactory( + config.getDriverProperty(DriverProperties.JSON_PARSER_FACTORY.getKey(), null)); } catch (SQLException e) { throw e; } catch (Exception e) { @@ -128,6 +136,57 @@ public ConnectionImpl(String url, Properties info) throws SQLException { } } + private JsonParserFactory instantiateJsonParserFactory(String className) throws SQLException { + if (className == null || className.trim().isEmpty()) { + throw new SQLException("Value of '" + DriverProperties.JSON_PARSER_FACTORY.getKey() + + "' is empty string but should be a FQN of factory class."); + } + try { + Class factoryClass = loadFactoryClass(className); + if (!JsonParserFactory.class.isAssignableFrom(factoryClass)) { + throw new SQLException("Class '" + className + "' should implement " + JsonParserFactory.class.getName()); + } + + return (JsonParserFactory) factoryClass.getDeclaredConstructor().newInstance(); + } catch (ClassNotFoundException e) { + throw new SQLException("Class '" + className + "' (implementing JsonParserFactory ) not found. Check " + + DriverProperties.JSON_PARSER_FACTORY.getKey() + " property", e); + } catch (InvocationTargetException | InstantiationException | IllegalAccessException | + NoSuchMethodException e) { + throw new SQLException("Failed to instantiate '" + className + "'. Check class implementation.", e); + } + } + + /** + * Resolves a user-supplied factory class name. JDBC drivers are commonly deployed in a + * parent class loader (e.g. servlet container {@code lib/}) while caller-supplied classes + * live in the application class loader, so the thread context class loader is tried first + * and the driver's own class loader is used as a fallback. + */ + private Class loadFactoryClass(String className) throws ClassNotFoundException { + ClassNotFoundException firstFailure = null; + ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); + if (contextClassLoader != null) { + try { + return contextClassLoader.loadClass(className); + } catch (ClassNotFoundException e) { + firstFailure = e; + } + } + try { + return this.getClass().getClassLoader().loadClass(className); + } catch (ClassNotFoundException e) { + if (firstFailure != null) { + e.addSuppressed(firstFailure); + } + throw e; + } + } + + public JsonParserFactory getJsonParserFactory() { + return jsonParserFactory; + } + public SqlParserFacade getSqlParser() { return sqlParser; } @@ -538,7 +597,7 @@ public Properties getClientInfo() throws SQLException { * Creating multilevel arrays may be confusing. * Spec doesn't tell much about it so there may be different variants. * Note: createArrayOf() expect type name be for element of the array and for - * Array(Array(Int8)) it should be Int8 according to spec. However element type + * Array(Array(Int8)) it should be Int8 according to spec. However, element type * of 1st level array is Array(Int8) * @param typeName the SQL name of the type the elements of the array map to. The typeName is a * database-specific name which may be the name of a built-in type, a user-defined type or a standard SQL type supported by this database. This diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/DriverProperties.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/DriverProperties.java index 4be3deb9d..8d6123628 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/DriverProperties.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/DriverProperties.java @@ -145,6 +145,14 @@ public enum DriverProperties { */ @Deprecated TYPE_MAPPINGS("typeMappings", null), + + /** + * Defines which {@link com.clickhouse.client.api.data_formats.JsonParserFactory} implementation the connection + * should use when the response is in {@code JSONEachRow} format. Value is the fully-qualified class name of + * the factory class. + */ + JSON_PARSER_FACTORY("jdbc_json_parser_factory", null), + ; diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/ResultSetImpl.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/ResultSetImpl.java index a0469dec8..62259be99 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/ResultSetImpl.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/ResultSetImpl.java @@ -1,7 +1,7 @@ package com.clickhouse.jdbc; import com.clickhouse.client.api.DataTypeUtils; -import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; +import com.clickhouse.client.api.data_formats.ClickHouseFormatReader; import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.QueryResponse; import com.clickhouse.data.ClickHouseColumn; @@ -40,6 +40,7 @@ import java.time.ZonedDateTime; import java.util.Calendar; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -47,7 +48,7 @@ public class ResultSetImpl implements ResultSet, JdbcV2Wrapper { private static final Logger log = LoggerFactory.getLogger(ResultSetImpl.class); private ResultSetMetaDataImpl metaData; - protected ClickHouseBinaryFormatReader reader; + protected ClickHouseFormatReader reader; private QueryResponse response; private boolean closed; private final StatementImpl parentStatement; @@ -71,12 +72,12 @@ public class ResultSetImpl implements ResultSet, JdbcV2Wrapper { private final Map> connTypeMap; - public ResultSetImpl(StatementImpl parentStatement, QueryResponse response, ClickHouseBinaryFormatReader reader, + public ResultSetImpl(StatementImpl parentStatement, QueryResponse response, ClickHouseFormatReader reader, Consumer onDataTransferException) throws SQLException { this(parentStatement, response, reader, onDataTransferException, JdbcUtils.DATA_TYPE_CLASS_MAP); } - public ResultSetImpl(StatementImpl parentStatement, QueryResponse response, ClickHouseBinaryFormatReader reader, + public ResultSetImpl(StatementImpl parentStatement, QueryResponse response, ClickHouseFormatReader reader, Consumer onDataTransferException, Map> defaultTypeMap) throws SQLException { this.parentStatement = parentStatement; @@ -1506,8 +1507,13 @@ public T getObjectImpl(String columnLabel, Class type, Map T getObjectImpl(String columnLabel, Class type, Map) { + throw new SQLException("JSONEachRow arrays are returned as parser-native List values. " + + "Use getObject(...) to read this column."); } - return (T) JdbcUtils.convert(reader.readValue(columnLabel), type, column); + return (T) JdbcUtils.convert(value, type, column); } else { wasNull = true; return null; diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/StatementImpl.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/StatementImpl.java index d5c4e4233..9bb3db82e 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/StatementImpl.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/StatementImpl.java @@ -1,11 +1,13 @@ package com.clickhouse.jdbc; import com.clickhouse.client.api.ClientConfigProperties; -import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; +import com.clickhouse.client.api.data_formats.ClickHouseFormatReader; +import com.clickhouse.client.api.data_formats.JSONEachRowFormatReader; import com.clickhouse.client.api.internal.ServerSettings; import com.clickhouse.client.api.query.QueryResponse; import com.clickhouse.client.api.query.QuerySettings; import com.clickhouse.client.api.sql.SQLUtils; +import com.clickhouse.data.ClickHouseFormat; import com.clickhouse.jdbc.internal.ExceptionUtils; import com.clickhouse.jdbc.internal.FeatureManager; import com.clickhouse.jdbc.internal.ParsedStatement; @@ -177,11 +179,22 @@ protected ResultSetImpl executeQueryImpl(String sql, QuerySettings settings) thr response = connection.getClient().query(lastStatementSql, mergedSettings).get(queryTimeout, TimeUnit.SECONDS); } - if (response.getFormat().isText()) { - throw new SQLException("Only RowBinaryWithNameAndTypes is supported for output format. Please check your query.", + ClickHouseFormatReader reader; + if (response.getFormat() == ClickHouseFormat.JSONEachRow) { + if (connection.getJsonParserFactory() == null) { + throw new SQLException("Response is in JSONEachRow format, but " + + DriverProperties.JSON_PARSER_FACTORY.getKey() + " is not configured. Set " + + DriverProperties.JSON_PARSER_FACTORY.getKey() + " to a JsonParserFactory implementation.", + ExceptionUtils.SQL_STATE_CLIENT_ERROR); + } + reader = new JSONEachRowFormatReader(connection.getJsonParserFactory().createJsonParser(response.getInputStream())); + } else if (!response.getFormat().isText()) { + reader = connection.getClient().newBinaryFormatReader(response); + } else { + throw new SQLException("Only RowBinaryWithNameAndTypes and JSONEachRow are supported for output format. Please check your query.", ExceptionUtils.SQL_STATE_CLIENT_ERROR); } - ClickHouseBinaryFormatReader reader = connection.getClient().newBinaryFormatReader(response); + if (reader.getSchema() == null) { long writtenRows = 0L; try { diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/ResultSetImplTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/ResultSetImplTest.java index 460def882..54f4d0624 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/ResultSetImplTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/ResultSetImplTest.java @@ -1,6 +1,7 @@ package com.clickhouse.jdbc; import com.clickhouse.client.api.ClientConfigProperties; +import com.clickhouse.client.api.data_formats.JacksonJsonParserFactory; import com.clickhouse.data.ClickHouseVersion; import org.testng.Assert; import org.testng.annotations.Test; @@ -29,6 +30,7 @@ import java.time.LocalDateTime; import java.util.HashMap; import java.util.Map; +import java.util.List; import java.util.Properties; import static org.testng.Assert.assertEquals; @@ -268,6 +270,50 @@ public void testCursorPosition() throws SQLException { } } + @Test(groups = {"integration"}) + public void testJsonEachRowCursorPositionDetectsLastRow() throws SQLException { + Properties properties = new Properties(); + properties.setProperty(DriverProperties.JSON_PARSER_FACTORY.getKey(), JacksonJsonParserFactory.class.getName()); + properties.setProperty(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey(), "JSONEachRow"); + try (Connection conn = getJdbcConnection(properties); Statement stmt = conn.createStatement()) { + int limit = 13; + try (ResultSet rs = stmt.executeQuery("SELECT number FROM system.numbers LIMIT " + limit)) { + + for (int i = 0; i < limit - 1; i++) { + Assert.assertTrue(rs.next()); + Assert.assertFalse(rs.isLast()); + } + + Assert.assertTrue(rs.next()); + Assert.assertTrue(rs.isLast()); + + Assert.assertFalse(rs.next()); + Assert.assertFalse(rs.isLast()); + } + } + } + + @Test(groups = {"integration"}) + public void testJsonEachRowGetObjectReturnsParserNativeArray() throws SQLException { + Properties properties = new Properties(); + properties.setProperty(DriverProperties.JSON_PARSER_FACTORY.getKey(), JacksonJsonParserFactory.class.getName()); + try (Connection conn = getJdbcConnection(properties); Statement stmt = conn.createStatement()) { + try (ResultSet rs = stmt.executeQuery("SELECT [1, 2, 3] AS arr FORMAT JSONEachRow")) { + Assert.assertTrue(rs.next()); + Object value = rs.getObject("arr"); + Assert.assertTrue(value instanceof List, "Expected parser-native List but got " + value.getClass()); + + List list = (List) value; + Assert.assertEquals(list.size(), 3); + Assert.assertEquals(((Number) list.get(0)).intValue(), 1); + Assert.assertEquals(((Number) list.get(1)).intValue(), 2); + Assert.assertEquals(((Number) list.get(2)).intValue(), 3); + + Assert.expectThrows(SQLException.class, () -> rs.getArray("arr")); + } + } + } + @Test(groups = {"integration"}) public void testFetchDirectionsAndSize() throws SQLException { diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java index eed401a14..7642f4e6c 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java @@ -1,6 +1,9 @@ package com.clickhouse.jdbc; import com.clickhouse.client.api.ClientConfigProperties; +import com.clickhouse.client.api.data_formats.GsonJsonParserFactory; +import com.clickhouse.client.api.data_formats.JacksonJsonParserFactory; +import com.clickhouse.client.api.data_formats.JsonParserFactory; import com.clickhouse.client.api.internal.ServerSettings; import com.clickhouse.client.api.query.GenericRecord; import com.clickhouse.data.ClickHouseVersion; @@ -848,6 +851,44 @@ public void testTextFormatInResponse() throws Exception { } } + @Test(groups = {"integration"}, dataProvider = "testJSONEachRowFormatDP") + public void testJSONEachRowFormat(Class parserFactory) throws Exception { + Properties properties = new Properties(); + properties.setProperty(DriverProperties.JSON_PARSER_FACTORY.getKey(), parserFactory.getName()); + try (Connection conn = getJdbcConnection(properties)) { + try (Statement stmt = conn.createStatement()) { + try (ResultSet rs = stmt.executeQuery("SELECT 1 AS num, 'test' AS str FORMAT JSONEachRow")) { + assertTrue(rs.next()); + assertEquals(rs.getInt("num"), 1); + assertEquals(rs.getString("str"), "test"); + assertFalse(rs.next()); + } + } + } + } + + @Test(groups = {"integration"}) + public void testJSONEachRowFormatRequiresParserFactory() throws Exception { + try (Connection conn = getJdbcConnection(); + Statement stmt = conn.createStatement()) { + try { + stmt.executeQuery("SELECT 1 AS num FORMAT JSONEachRow"); + fail("Expected SQLException"); + } catch (SQLException e) { + assertTrue(e.getMessage().contains(DriverProperties.JSON_PARSER_FACTORY.getKey()), + "Unexpected message: " + e.getMessage()); + } + } + } + + @DataProvider + public static Object[][] testJSONEachRowFormatDP() { + return new Object[][] { + {JacksonJsonParserFactory.class}, + {GsonJsonParserFactory.class}, + }; + } + @Test(groups = "integration") void testWithClause() throws Exception { int count = 0; diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/internal/JdbcConfigurationTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/internal/JdbcConfigurationTest.java index 9e0abe8bc..917750f78 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/internal/JdbcConfigurationTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/internal/JdbcConfigurationTest.java @@ -126,7 +126,8 @@ public void testParseURLValid(String jdbcURL, Properties properties, { JdbcConfiguration configuration = new JdbcConfiguration(jdbcURL, properties); assertEquals(configuration.getConnectionUrl(), connectionURL, "URL: " + jdbcURL); - assertEquals(configuration.clientProperties, expectedClientProps, "URL: " + jdbcURL); + assertEquals(configuration.clientProperties, expectedClientProps, "expected: " + expectedClientProps + + " actual: " + configuration.clientProperties); Client.Builder bob = new Client.Builder(); configuration.applyClientProperties(bob); Client client = bob.build(); From de14e27ce69235ddb43fe0e94aaaf747241fda48 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 10 Jun 2026 11:01:31 -0700 Subject: [PATCH 2/3] Added gradle wrapper in example project --- .../gradle/wrapper/gradle-wrapper.jar | Bin 0 -> 48966 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 examples/client-v2-json-processors/gradle/wrapper/gradle-wrapper.jar diff --git a/examples/client-v2-json-processors/gradle/wrapper/gradle-wrapper.jar b/examples/client-v2-json-processors/gradle/wrapper/gradle-wrapper.jar new file mode 100644 index 0000000000000000000000000000000000000000..d997cfc60f4cff0e7451d19d49a82fa986695d07 GIT binary patch literal 48966 zcma&NW0WmQwk%w>ZQHhO+qUi6W!pA(xoVef+k2O7+pkXd9rt^$@9p#T8Y9=Q^(R-x zjL3*NQ$ZRS1O)&B0s;U4fbe_$e;)(@NB~(;6+v1_IWc+}NnuerWl>cXPyoQcezKvZ z?Yzc@<~LK@Yhh-7jwvSDadFw~t7KfJ%AUfU*p0wc+3m9#p=Zo4`H`aA_wBL6 z9Q`7!;Ok~8YhZ^Vt#N97bt5aZ#mQc8r~hs3;R?H6V4(!oxSADTK|DR2PL6SQ3v6jM<>eLMh9 zAsd(APyxHNFK|G4hA_zi+YV?J+3K_*DIrdla>calRjaE)4(?YnX+AMqEM!Y|ED{^2 zI5gZ%nG-1qAVtl==8o0&F1N+aPj`Oo99RfDNP#ZHw}}UKV)zw6yy%~8Se#sKr;3?g zJGOkV2luy~HgMlEJB+L<_$@9sUXM7@bI)>-K!}JQUCUwuMdq@68q*dV+{L#Vc?r<( z?Wf1HbqxnI6=(Aw!Vv*Z1H_SoPtQTiy^bDVD8L=rRZ`IoIh@}a`!hY>VN&316I#k} z1Sg~_3ApcIFaoZ+d}>rz0Z8DL*zGq%zU1vF1z1D^YDnQrG3^QourmO6;_SrGg3?qWd9R1GMnKV>0++L*NTt>aF2*kcZ;WaudfBhTaqikS(+iNzDggUqvhh?g ziJCF8kA+V@7zi30n=b(3>X0X^lcCCKT(CI)fz-wfOA1P()V)1OciPu4b_B5ORPq&l zchP6l3u9{2on%uTwo>b-v0sIrRwPOzG;Wcq8mstd&?Pgb9rRqF#Yol1d|Q6 z7O20!+zXL(B%tC}@3QOs&T8B=I*k{!Y74nv#{M<0_g4BCf1)-f)6~`;(P-= zPqqH2%j0LDX2k5|_)zavpD{L1BW?<+s$>F&1VNb3T+gu!Dgd{W+na9(yV`M7UaCBuJZg1Y)y6{U}0=LTvxBDApz@r>dGt(m^v|jy&aLA zdsOeJcquuj3G^NkH)g)z@gTzgpr!zpE$0>$aT^{((&VA>+(nQB!M(NnPvEP}ZRz+6 zE!=UW!r7sbX3>{1{XW1?hSDNsur6cNeYxE{$bFwZzZ597{pDqjr%ag85sIns_Xz%= zqY{h#z8J6GA~vfLQ2-jWWcloE5LA62jta=C*1KxAL}jugoPqj4el4R4g3zC4nE#2-NeS{c3#!2tIS|1h8*|kpw2VSH9OcIQZx0Yh!8~P&p}fI$4Bj9Z zr5Yv?i-PfO#<}clM>mO(D0wHniZZdv8pOuJFW z+-u}BH84PQCgT~VWBM88vtCly1y$uEGJ<7vnW%!2yV>l>dxA0X0q{cN6y3u$8R-*f z-4^OlZ1HmxCv`dFW%quP<7xzAbtiFxvY0M1&2ng&A}QXAVR=prc_5m(D+_?hv#$M^ zG#MQ#fHMc!+S%HgU^Qv7Z9eu6eNqpSr3e8(;No*YfovbJ;60LjCzv9O~^>gFKO>t zGZg9`a5;$hksp*fHp{7&RE@DM&Pa@a>Kwk%*F7UGO|}^Z0ho1U$THOgX9jtCW6N$v zLOm}xcMBtw)CC(;LLX!R9jp|UsBWGfs@HaMiosA3#hFee7(4vLY}IrhD++}>pY zo+=_h+uJ;j^CP*OGQ9$0q+%}UB`4`5c766d#)*Czs<91wxw)jI^IdvyjT%<8OqI=i zNn0OUqW#POg^4ma)e2b?*Xv;dri*N0SJ7_{&0>;S!)!YV1TQuiT1C3ZFDvThe}yTCmErx#6yyQ4X@OAbHhdEV!K2%;7J>tiUZF)>Z|eRVDwtDC~=J z*M8|WEgzsyNH@-5lJE+P6HrurgY!PqtWk z^69SOHZ*}xn|j2FDVg`qRT}ob*1XiGo=x8MDEX)duljcVO}oJjuAbB$Z+f&!{z3k< zO6+{@O#2^s4qT`6k}Nw?DKV1DU~}0jVA)(kNz$c-p`*FNG#Gb&o?ko70F||R^y*hD z6HD|hJzF)G&^K=vuN$@b2fIfHVFw@hC_-0hPnB!1{=Nn~ran4VeTMM(Xx2A3h95U} z&J#Kw4>*V(LHOA<3Dy{sbW-9k5M2<%yDw~ce0+aez8 z04skG8@QEESIL;m-@Mf_hY!)KkEUowHu(>)Inz(pM`@pkxz z1_K#Qs6$E^c$7w=JLy>nSY)>aY;x2z`LW-$$rnY0!suTZSG)^0ZMeT#$0_oER zfZ1Hf>#TP|;J^rzn3V^2)Dy!goj6roAho>c=?28yjzQ>N-yU)XduKq8Lb3+ZA|#-{ z?34)Ml8%)3F1}oF;q9XFxoM}Zn{~2>kr%X_=WMen%b>n))hx6kHWNoKUBAz?($h(m(l;U*Gq7;p5J{B;kfO^C%C9HhtW!=O3-h>$U zI2=uaEymeK^h#QuB8a?1Qr0Gn;ZZ@;otg2l>gf= z$_mO!iis+#(8-GZw`ZiCnt}>qKmghHCb)`6U!8qS*DhBANfGj|U2C->7>*Bqe5h<% zF+9uy>$;#cZB>?Wdz3mqi2Y>+6-#!Dd56@$WF{_^P2?6kNNfaw!r74>MZUNkFAt*H zvS@2hNmT%xnXp}_1gixv9!5#YI3ftgFXG20Vt1IQ(~+HmryrZI+r0(y2Scl+y=G^* zxt$Vvn&S=Vul-rgOlYNio7%ST_3!t`_`N@SCv$ppCqok(Q+i_?OL}2@TU$dr6B$c8 zQ$Z(lS6fp%7f}ymQwJAIdpkN~8$)O3|K7Z;{FD?hBSP-#pJgq0C_SFT;^sBc#da0M z;^UuXXq{!hEwQpp(o9+)jPM6ru1P$u0evVO(NJ;%0FgmMNlJ+BJ zf^`a|U*ab?uN*Ue>tHJ$Pl~chCwRnxi3%X06NxwlIAKa*KReLL^y1B^nuy|^SPj3} z5X|?1divh3@zci;648jb2qEOm!_8Tjh3gi;H%2`d`~Q(IL{Wcl1C18+&P>tU&0!nO z&+7mpvr2SsTj=@sX zxG=;T^f7Rg=c=V*u8X(fo)4;RYax^+=quviOJ{>r6{wgf)g){I&qe`=HL}6J>i6Ne zSZ*h9f&JG>Y`@Bg5Pb&>4&UqFp9I<8o`n4W_V=4AugM`RqUeS-!`OyNLyKMqa_Ct| zON-hyk#-}{lZZx>B1F@dF^8S>x|C*QAjKqn&Ej9H#z@Q#KA*ckBX@^;gIP&?aK15l z*EY@kG57oUcm(d{NyXg6$Kj#xR5XdZ1EBCT+Zy!gyXwN&b_zI&$$>7R#{ zh8U@H8NY-cA*CBfH$OCs^priPwtwrzFjDO}DBn#mgbI~hn}cp2U{yv@S)iy|jR9+E zgd(hF|1cyC#te0P;iFGqpNBqc(k<{p^1>wHE_c8Tr4|&NV4mzpzFe;Cr)C~qpVNjl z^u(^s5=kj{QBae)Y*#^A39jT4`!NuIUQzD#DOyfa!R=PrX6oS@x@kJV)Cn$!xTK9A&VI#F-Slt8I4|=$bcjaC5h=9E{51g8X5q1Qfg~~G>qAgy*7h4-WuqE zlIEx?Hu*%99?$6TheLAD4NIMO=Q@*;gaXDl6yLLXfFX0*1-9KQm42c%WX*AXFo$it z?FwnWn2tBHY&Qj6=PV?ergU$VKzu+`(5pCRqX}IoSFo?P!`sff%u1?N+(KsoL+K={ zi*JGl%_jiuB;&YW+n%1o^%5@!HB9}OlIdQZ*XzQ%vu!8p2gnKW+!X>@oC{gp3lNx^ z82|5Jdg9-B<1j|y(@3J;$D-lqdnf0Q6T~q7;#O}EMPV3k(bi$DpZwj9(UhU%_l&nN zR}8tN_NhDMhs)gtG*76~+W2yQ{!kDTE@X4gft2?W;S$BLp9X z;sh2jpm!mkfPX>Vuqxyt76<@f4fyY%&iuDfS1@#PHgzHqG;=X^`X}t2|Alr^lx^ja z1rhvG(PH(a0THitc?4hk=P*#IS;-`fjOKqJ4kgo@dAD@ob*))H)=)6s3cthp&4Q55 z4dQRdG0EveK*(ZUCFcCjILgS#$@%y=8leYxN-%zQaky@H?kjhyBrLYA!cv>kV5;i1 zZ^w&U7s&K8fNr4Pfy9GyTK2Tiay4Y_PsPWoWW5YA8nfUkoyjU)i@nKj@4rY13sxO6 z_NzYdG=Vr<@08Xi#8rnX&^d{Bl`oHXO6Y3!v2U~ZV>I*30X3X&4@zqqVO~RyF)6?a zD(<+33_9TqeHL)#Y?($m4_zZvaJXWXppZ4?wo?$wF)%M6rEVk2gM=l9k+=*Q+((fI zIUBH6)}M?ahSxD4lgmJ30ygk#4d!O@?%WNEONommx`ZK81ZV)mJpKB`PgQ}F>NGdV zkV|>^}oWQd6@Ay7$&)6!% zOu_p~TZ3A#G_UqiJ85&*$!(+!V*+*{&-JXb53gtc9n3>8)T$jUVXe+M6n$m633Mi? zlh5{_+6iZ<%gMWMrtHyDl(u-hMl^DViUDc50UD;0g_l$F`Hb(F=o+?94B0fjb;|?Q5c~TWX>t8i1RP@>Ccgm z?2=z0coeb?uvn44moKFb^+(#pAdHE7{EW(DxJE=@Z0^Am`dpm98e`*S+-~*zmhdQ7 zCNig0!yUu5U#>KKocrg-xMjQoNzQ`th0f{!0`ammp_KMFh?_zF4#YhF35bPE&Fq~_ z#VnniU6fso{!3Z^1C57q?0i!ok(a zL;-f$YlDk%qi%n637_$=Gw=bBY}8#meS~+#X}Oz~ZKd%q(UE>f%!qca?(u}) z!tLTuQadlAN;a#^A?!@V=T?oeJ1f7yRy)H1zn_+wARewYIYr`zD=^v+D|ObvH4rOB zT@duqF>$Dk6&i|pZh?%Wq-7_kyP4l)-nqBz#G0lqo3J2D%zmbU)>3)5e?sTZy8|~B zPC7!`eD+deR?L6$6 z-e{!ihef=f<4HPZ9rSt&yb=5Q)BFAXWPR^~a&Zru?8146wvlm;<)ugbd|!}O6aE0t z6`#KqcH#S#*yz-K90+!Fhv+ zKH+?!_0yl|gWXSaASLcB9a8g7i%qz*vbO)YW`Q@Nxpp*6TZ*OO8Z|5-UWihd@CUXF zY!aTAZ$c^?4hiaq34=s2il}#Pxu=#c2^=(PbHNAyUqy__kR+n?twKrQe^8l6rk=orf}Mk80viC1NZ^1q zeF~g*iGp0=jKncK%s@#jZcn6=EiR<8S#)yiEOuwbG;SV$4lB^R?7sxOf8)oq$sT)) zA&nBCFJxsnci+)owdCHV#cjP2|1j22xIRsxHrLLBk3GI|OppUv3%r>#;J|26!W>xC z9gq@NQWJ`|gH}F{-QG#R6xlT<;=43amaDT>VaG*;GfPZJ&W*rO8WAQQc^JGw-fz-| zzAe&RAnC(gAP#FoJtt~ynR3Z<)m_<9Oo)XW}CWd50^eI4!1p4}s(zLhBIDi5r zr{UH>YIz2!+&Cy(RI(;ja_>SUC2Q`ohWPlI+sK-6IU}*nIsT)vLnuVPFM%~gdel}S zUlY%>H$?-rQRGTdUM^p^FEkqnwC{^BGl|gM)h9zkXplL90;yOcgt(8&LJwOj!5Qgy zu$@^*k%9JoAzwj@iSB^SNu#YVl@&*g$uYxxsJBvIQ>bfuS97JccQcS7&a z)`1m2^@5c9pD`P$VqH*O*fxkvFRtH-@Pd0@3y2!jW>i=jabBCJ+bW@wwUkWjwx_WR zHH5*XR4hbQ1`D@4@unmyEX)!?^~_}~JQNvP4jO&F)CH9srkFhf8h*=P z;X1&vs_&v03#BGc`|#@!ZONxVj9Ssb#_d63jxA6dX_RBt(s;ig3#s(YU3P3klF;mc z%%@^IJUAlGE=cnsTH+(qb1SxN@HzfAjYcUCb(VU)JV^3ZC;#k!t?XjaC!|68eLE zU_hlvOSNj7Qlr{x)y$S$l^2DPCMA=pzapcSkjfk*r!iWU%T{?<3#Hw6s1ux1^Ao6o zR@5DIfo-|c9AaFw848Y!BVG-+vURe;I29F#hLu$9o}oSa9&2sgG#;lj@@)9|2Z3 zon?%NV&AYSVnd~eW~v0yoF$X^1FR@i2kin0mFLG8-aA>hYK;B%TJ~7%P4?_{Bu<0t zvmI)Uk-MRncVb)A890>OqnYf=wu-J5A~^%4jpK~*xp)=h0BZB4*5uWrP>iRV+|kMX zv+BEskY~(P-K)-!JSHR`$brY)HFI|L@YyrxheT3cgHu}KtF%s%k3B`X)E_lA=E>M4 z2VV3M{c0*)`qZAsJ==)F#D~2Ndzm@hKhSBL_Sf3{ctckh-rB`gkfC?Dp6FdM?p;vv z#UlQMp3H5*)8o#Ys@-aj7O#brUfgQ7BjG`7 ztoE7v-tH2%KVC$xKYf%uvZD!_uf3x>h?8r!zYHkcc7$Gdn(6cDmYL&p3pCfaSfY4$ zG|yuujr6!Wl0}V%* zQ;nY##kEdvo8YY=SVDb)M>^Ub9e#4c$O&urD$uaRtxm-UH=6_s0m^^5y^_+F^Q?;8 z+Fd?+De}er^2EmFNn&e8SyS*`*`e;KFIG&+x5iWCsrEyH*0SFBCMx?`m5~hl1BrT> zr8W3*3}Fwsx@%UOuxNoCSoL%AM{Uj|v@>l{pYYI&D$j`&**;?X`cuOOk~?;U{~xvDUjaiH^d`A+gQL#Z?*lm)x_n6R-S% zf6*=Q1m>mq5|Niefl8s=5F={ncn5S;6~&Ns2)yGZ@wt&u4c+)Sk?hdfI^b77@K-=y zM_k=j5hp&u`2nkJK+2Lw`uLypr4dO?Bm3BTZdtWnQa5unCoTKIiG81t4bG`epBU5| zG{toT`)LE}&j{P+AFj`YZrjF-^>k+`zCM`QcQz^Ba4BEte@S}j=Q_Opx14jq|DB}& zNB44BOJ`?GJM({v`gh9pzbg8-%Un=E@uLfJwGkagLEM^!`ct3s5@-xqq*xd+2C@eu z*1ge`retZK)=bPO<`>@62cLN?^S%v#EsiPQF`cg&I7{}l?)}O$!^wNJp4Zd;1yBbQ zv@_7x7d6aXJvGHkNNcOg?A};m_Nq7H=(+zqf9)e3&yP^EU63Ew!NW4CYj_!=OTVb* z-ijSrv0M)u=MF=@+`3ldT-hzOn$Ng><)WL0vqQ&jH>W7EmLLQY+c?%i9~f_x&{OYX z{?kyyNZ&gT*m$(%-OeDAJeC^c)X!k${D*c;c}9)0_7iWMbfu)!j3+{*!Dj|?C`sGz z2xWha)#`9@p*{-X2MN2a;%FM-WqB2h)GTqQH$ZsGD#Wi`;+$i?fk;23fLpYI^3TT3 z5+Zn3cu-_2Ck*@%3^L3}JpVN`5ZJ;gmKn>gm(Z)b%!v|RYf(qrmGL#0$WHQFw4mJqQ85w=$tn^7(z|eJ$3R0} z2k9^EU<^-$ygq!ZR+7wT0KViK8qkAO7xs*e@1dq{=M3haulHwA0~BYNytr7k2K*(W z755P9a^;Hdl2X;K{c}yWr|QH?PEuh6x)9n{^3m2QUfC_Q*BW&<9#^ZVwOolx@6y9- z-YF=S;mEypj68yxNxfJ56x%ES`z-5$M${V1HX(@#R>%$X`67*Ab8vC6UzvoDOY*P= zFbPXany0%>rqH1gi7d>e`=PWZTG>^=#PQf&iJjJ0&2dO(4b8) zCl%8xJg1mg4__!?t|y_roExn~%u@Eu|p9YFb`8_qP@v#KW#kFs4eVetJ+Q+s|Y0?#D z@?dt_BA7C4tGpjOB~*LFu0!5oU(_xj7xA$meN)Z;q4Z_Rb7jY1rJBzJPr0V=(y99F zh=V-NbK+64rd#ltw~7X-%kP$R896DxRuj)p7Zj@8&>IlP&}ME3s9eV2R>SpUnSxeg zmpm?HQJ^u1T;pvwvlc4F_)>3P~jlTch4+u6;o{@PtpnJcn~p0v_6Po%*KkTXV#2AGc) zv)jvvC?l#s$yvyy=>=7D3pkmV24xhd7<5}f_u5!8gmOU|4555dv`I=rLWW!W!Uxg| zFGXpH3~)9!C2|Y6oB~$gz(;$CTnw&R&psa+E!KNgrE1+WkLM6SOf$>sGW+Y{>u?Fw zTc!xG{pa3c#y@d$d0e7a9~e_xjGcaw5f6Fk>lg$Jm}cFd%BO_YT(9s+_Q;ft%1*k$ z_cXkf&QHkaQr9U?*Gr$r6|bCV>2S)Cedfk3rO?JbyabY zgqxm#BM7Sg6s-`5%(p@SxBJzR6w`O6`+Kuo36wwBzwf6K{0HENVz^^w|E$r zdZM%T0oy8OK|>>2vSzw5rqoqEroCZ%(^OmOSFN84B2-8Z?R1)Pn9|5Xkui(fQRl^zA35EH^(JbuQd@Uh z2FJ6C(5FDD(++_NLOG)1H<+X~pt68d@JiB8iUQSZ+?qc;Jr+aJ8bKF3z`K&zSl&C7 zEgl&!h?sc=}K7 ziEC(3IrY?h7|d= zVjh{@BGW^AaNcdRceoiKmQI+F$ITdcM$YigXtH)6<-7d@5DyyWw}s!`72j`A{QC~e ze-u0a6A;QSPT$vqf3f(kO1j^%GYap*vfWQ@X=n{lR9%HX^R~t+HoeaT5%L7XSTNn` zCzo})tF@DMZ$|t6$KTx+WQqu~PXPa9FL&shBGx3C>FlGz}7gjfv}(NKvjR#r5PL$a1>%asaylWA8^g!KJ=$}_UccHmi zAZd5c{I&Ywpi3a1#27C6TC~zm3y8D>_1an8XHGNgL?uT$p+a<5AdWLR6w9jdhUt9U zz?)93=1p$x;Qiq!CYbX&S}+IITWLkfu%T6X5(pk9-fs8lh9z8h?9+>GlFeFcs*Z>u zJSaL!2?L8LbOu_Ye!=4~ZKL?643lcsNn8>qUT|q&Rv+(z>Z9=tyG&5}zZK&Q?S!nG zR;Ui^<406=jLYA>zl!a-OXH#J-pP4A`=)r%9HV5m1qGZ1m*t^wi>3$JRcH)3Q(LQz z(3}~y3=QsUu!PN$$N~#yBP@=aJ+Bkp_hx8^x1Ou6+(Kk9l1CXr4p~IQvq@AUePuAj zcq5>YDr(JTmrAuLwn6sgohTR-vc^y^#I{grF7 zg}8?&5!^$|{X`C;YrZ7?rKH#`=n0zck(q37+5%U;Hmds2w+dLmm9|@`HqQ<5CUEz{I1eNIL?X~rd{f71y z>_<94#1G+j`d5|fKK@>QDK6|HRR|9UZvO6HdB1afJvuwUf8bw>_Fha)Ii8I}Gqw}p zdS~e^K4j{d%y+A#OBa1C4i0)sM=}tjd8fZ9#uY}{#G7rJp{t6?*5*A^KKhim06i{}OJ%eA@M~zIfA`h_gJ_o%w;FaFQMnVkBT|_ z(`m9r+11~EPh9f7>S=$F7|ibj=4Pt>WVzk6NfGRvI_aG66RHig-(S%WKRLP%_h0He``xT))N^RI@6!ADl=*vsqVb|7 zr~Lwl6qn|u!%is<{YA`Mde2Z${@EAHC^t>4`X;F9za=RC{{$4OcGmw%9+{$i@!cCn z;7w~r8HY->M@3OzYh+L7Z2Lc8AcP*FZbl6VVN*_sp}K zQP|=g@aFthq}*?|+Gm4@wbs_?Fx-HD2%)_UDJ);X88~7ch~d0cJ!<7;mv>iv!RS$a z;(-cYTW=K=|F0gIg3EW0%u2CSr(Kx}yLoki|KSIt$#P(O!=UjBGRzb3L3-?NGr7!! z^VC7_Q(GhT;C*(bLivfhlRDVdz7=h%ABuLA2g$qy)A}U@Kj_L-Jd|--fy#-*ESRo| zgu?*?jGEgs9y>1`t}|^Ucd1I=1N=mOo{8Ph zwZS(F%G?nfI{#%sGayNItK9J5P)Qk+^4$ZoXZJ0G1}hwcckJ0g-QJ<)3%`bF8}(ahYIjKFYMtg3X;e7J18ZvDkV@N=nxvDl zo?}lXoT3pZY;4$QKI`~GFuQKv;G6b<8;o89Hd2yu+|%sU(9C=h8ibwZ zARqZ#lk@kp4*#URe-YmpRc&=-b&QP>5b{9{(tH*)(@ZPKfOslBgwCPx6d*{XMX|Q{y0F!5a^ScCE;h8bQmTJR3*}A>aGcDF0?tU)Tnml z#DgruwAva-fiU3s*POY_ZHiJyW%v+733X`&ocwHz$uqJCOhrM;#u*V2eK$D5HiN(` zII{BEg(PV6#_Nv3rZBUyd+TI!>L72KW_Oml6L=pNv#aOl( zgpYxAH^@2aJQu3urlrCeanwSpHHD_Cxb+=cm49{ZU5Z@;{^{okEJ6&fpDD31w~$`% zcz@_REsC~Vq>3YF7yJ41ZEPBW&%|OwlnfG|QNpiX;fGR0f^3?PEf|-33P&LFGe`8^ zaX3M+*h+?6;s|=$j*d|S-r6PSHnmLqm9oshPNpGzlxV21cFrxcQLidd2%h>n%Mc4{ z|JWBvtbb;(-nhWpPO95hR>(e(H$n%*pCh0k4xE#I%xu=#B)zXSaH+azwCI;0@bY<*-10-Qyaq%5NxSlq_@YJUUwy z*d;qPjW^cuKxdXiOWwP}5FN6SZW~NqB%4?|WifPNZr&XNVkzF0n#Y)pbaEodqNO4F z2Bq#^Gr^Ji3!T9`_!D;a1lW$?!LQ-iYV_A{FQ~^C-Jp`_5uOC)6+mzBr4Nl3fHly% zcXeU3x-?#J`=p$6c~$T~V^!C0Bk_3#WYrtoFCx9_5quCQ*4*?XG0n_9%l_!n`M85^ z7}~Clj~ocls6)V&sWGs?B<`{Ob>vnbXZwdda%ipwbzOJ(V`W>KBF5zdCTE8;mc&xU z^clCzd0(T#8*(})tSYSNP1N{FnNVAU^M1S_pq4VEQ*#5nv`CoYSALMEB zf6egyuRMzK2?r^M0hCD*sU;On6c0^Vh|#tRG*n1p5R)QyVw%Va37nMSV%9&uq^hp| zCHeu}y{m=NsA=naDy;q`fd9t)I$Qd-A1Il$#0KyDc>X)hKJViqNB{HnQyf5D(ZJ*J z{-oGB-%Q|QZ%Pqu34>fCy)Asi}IY7luNR9ebgH4DAjCVvSWfa%PE16 zkC7EIuEK}?IR!jgP%eX%dcxk4%N!zIjW4wYMfIq@s%GetDs^g!^p}DH46EP`Nh_wD z4Rwc4ezh1U$Mc)Fe6ii6eD^*iB2MFp-B-HhGTR0tC2?bq$#^J!v1r+Z0y+& znVub*k=*^0yP(c#mEvX}@Abx%&}!W(1olcWEHAVgskbBrzx(f2v&}4~WkVN?af#yi z4IE-(_^)?4e3(d{F@0<~NV5|e0eaB!?(g%l&Hq$UqzC_Enuest?CL+IrSD`tv8|{C z=79vnL=P6ne+}6X1&cd$kam=jCcv`~^y#R{doTh?6D?H)^M7-P+=D@?H;bt$*V+)K z?+?Ex3Z@8JE3c4eHDYItB^tSot;@2p_fuZ8mW^i^a(L;Xn6K+1GuG0n$v(38;+<78 zC?eMzbQCW2%&;U>j}b>YEH5>RkP44$QlG6k(KwXtq{e#13wnx5Jh=uH?lQIl8%Qxr zq%pDC)mYYKa?N>%aF%YwA}CzV@IOV9&a81d9eiU-6F&lGvz68~%{&4LuwV_5{#km3(tf`fejjs%`{Y`|0p!6|-U z8XQA9Sl=*kM|(2KA!LWOCY3Qq4sZ7r&}__rR*Sj(9W8R1_RxI&4TI+_7RSJF&-363 zJvczH?1(`Jb+RDJL9$Whnj8qJRI+Mz9=Qjvubb=Lz8nWVXG{Te;$%s9-D#$)-!{~w zIM(vkr#OM>2F7W$$Lq%fEYl%e|Tsc>9rB9c8 zQoi4nXomx3&sBI9AwaHkoOp%SMDf2@T#73Bi?|!r!Q?wc(^b_u4ranezYx~=aRV-a zD|_WPK^iJh&=)~h{t<>_$VMXsee;{r-|`#H|1?DZgWvuc*!&C2*(yv(4G5s{8ZRzt zZMC~5gjiU@6fPGMN%X~pL};Q`|IfPfs0m9;RV}xSxjb)*gmvGO1`CQb~W1M1{KwXBLyPz0JQG=JkVX zlPq&zNZS59gf-?*5Z0IFitTX4T$1Oo#_~V%4q2vI?Y@UkSHh}H9xZ1va}^oBrCY{+ z3wwj*FHCsS2}GdSG7W(|k+MWu9h1Qs6cft~RH)n*!;)5HmPX1DqrJ3-Cs%i4q^{$N zC&skM7#8f{&S!9Eq-WqyY$u?uTgrSDt#NU%{3bQZtUSkUof4`Z1P8aLOKJ+^dKh%n zfEfQ zO|P*J>;{=`9@D)qpnt`#NH>}sir*&oFC+W!HR)ecHcPwjF-|)}8+tR#@A+~CLl+Ab zCqp+=Cuc(&VGC1ZYg4CxIXYL>33p^wjIWJSh6R=oq)jD52q3~KVGt=w_z(arS!gx^ zSd|?!rzDu1$>0o0Y0+!iZU=ew^Hr+cq(I(C>9}^sBc++0+S#I;js@_NLD9>MH(tN3 zE5F+J_bYdPfYm5%7-e=lm?!-xlvX~nDkBqu!Zf0ra65JD&@tYDW+c@P3W-YyWe4^6 zhW?FUJ;c{^?b`N)03>!@#JI)r2&!6An27q?*^wyUx3T4uyeIl4*(4CV5OTK#RSnYt zq<+RKCdrYIJtdmNC-NtfH)K&pytbM^Mi6JWjkzJo0TdX>HOjJaIQmQ?Q;l2)8oN@d zVyT=%y@TihQaJX7#B2wY#_ufuaF55-sWO{OwUx$2zRyW$YM(CFBs4Y;YmBk(4u&u- zEf@rIR~4#}IMeq$?T%z3s3RAR7m%M?8No;a=1HXKP?ia#uwy!`4v0GFSjZiMii@ib z#xRmA-v~CSVl8z9cEWVEk;9_BKPS6Y2|bk#PAb|}gPxHs-dt*k`5tU#FZL)FLodY8 zmb!m`DagEJ#q1VKwO~%zmw7;LESf5u!KJNm829pbY_w$P2}16`Bb?0uoL3~V71;_U z`B~wKOB7Bp!Vn!M@o?RHydmah!dHPaT`&idV83kQPxA>E=~YgJC<)rdM1#B$JIgnq z0V{p|Cm3eeMaO58Wrv^9-kAOJ+*HR!;;A9z&>78VsYmF9$U^*ZE=K%d7=MZ~G?~Hz zSHlKWK!Us^%?uE6`E|_XI+nC354jkbUPvedHbh(DkKGkquYf}=-EEB1g>RC{O9ORL371y8V*CR5EW z@lmFq%MWEBdeHR7%(Rpf!Yg52vX%D7#@*^M`fy7Srb z^Ta9wcwf$89uL61@qeg2vc&TAGKSLV>YKI3#5lfs#q5Zm`~Ogef!!CoWWyiA=J;js z%X_n!njeF2MZgaVoMh@S@8%lR)AsYyzmqkj+C8ghxI4G6O7ovK$udULO!2$(|__`2~6JjuoERet}kenJ%I0pU_O@tU*Fsd4gm&hV?p%Y{!;r}{S^Fv z_4EJbVjFv7>+dE9{rBS@8&_vbx9>4!8&g4JV^e2mSwlNR^Z&ujriy)b3jzqfYb35o z!;J+c>%LY+?P!IticwSrP;x2|k>j3Sxg2X%E2%57

`Lem|V$A>eR0uN8Y&sdjtu z%-lD<@61@6?qUPjUg|mF7!P7`hx+st`i!^L7HVHtzwnM z)LuOANIzT#9tU4)C^WIXhZWqrO;jr_O5aErkklzt)R-JmAh8xHMJ>x>OvTiuRi}FY z-o@0kFwwl7p|ro=*2q*cFRX5GCq-v!LPD)Sq+Uz~UkOwx-?X&!Q^4H)$|;=n9{idC z0mJl`tCTs3+e_EFVzQ}s`f_4fijsucWy5y zarHoT>Q06Z4yI1RPNpW`@4hSzZT|J`MU3i(GqNhm*9O@MndJ{31uA^i zXo&^c`EZ}5W)(|YMl##@MuSK#wyZ3dwJEz*n@C(Ry$|d`^D=thayXFqxt*WW&sWdI zdm1wv#VCKa<7d2Qc#qzvUvivhK5wq*djL7Wqjvf}-c~}d#G)eG`(u<`NGei`BFe4Q ztTSs?Gc8Ff%_5T4ce&J0v*FT`y_9r!Po=sPtHs5~BlV6VEUNzxU+)+sX}ffdPTRI^ z+qP}ns9yQgjY^t0ddMx1Yd`|OB{sHnUC-B;qum1|`tR#P_@llx>d z=qpNN&?nZib(t90A9F*U%1GbB+O;dq!cNgmmdCrK=(zS1zg*9(7VMfv)QMkt_F=wz zHX2p4X-R*=tJI4A)3SrL`H^peBNHh&XC#sVR3D zt17qeF>BaCZNlQO7n@@BuWs&l(FtRjaVn~wW^x-GsjpFH!ETyl7Od{Wf;4=bzL5nj zW9c^ZodMnN{3Jkz2j2;qhCm1ede*6891vR9?(Dy)N|iENw}HKLIOrjB0x)pEs-aS{ zZR$tEyZxbP(;(l43^KjRtSuirNmw~Bg&6p;)vqM*>S#L>0+Pw5CU%4@&)8OX2ykYQ z^f^hk-5%!QzuzYniL*1Gs#S5Kp_*ld1EAmkInP+^w?#(?rbC2Bm&0c5Ko@6`_ zi!Nvd391nu^@AmpZ$_0fPR2~kQGJS7lSGwA7U>s@+!d_`(P5y;MT#U~_ONSo9d+bf zVj6MgWN=|%#Qn;vl*TNLE$Mw|*89{yJ=WN>j{?T*vqa$U$2_dg46R)8wl&CNS&iK{ z>HDBC9e3b3roJd}gK!T>takKP);KLj_9T;%knG_fN^S$4hb`E|)qy__^=mm&Z{~CF zhc*PxdrJ@xRkQ-8lbh3Ys@2ZaR)Q3z**-VSgeMHE>c5AH1bpSUor&dgTiMd5Wn|(# z8Rwb{#uWZG(Jo0co98|mg5zF}M*d>gAg|Zdex@}Ps&`51({MmNyHF;GD4EBT`oP|X zd=Tq9JYz*IP%@2oujruVrK#jAT97|%ww60Ov2He^5zA4)VihJ$-bxoaqE7zU$rmK) z#O!xp&k$!TOEiC8+p6`Q)uNg4u8*chnx*aw=#oP~05DS&8gnL>^zpBkqqiSQA{Ita z%-)qosk1^`p&aB@rZ#)&3_|u{QqZO z{f{A3)XMprL}2{=pM$*`z*fY;{=4e=u7&=s+zI)ANd+V!L%#^2hpy@#N-WbB%U2Zl zgD_E0AVVWdMiFi_u2qqxeAsRzD%>l|g-|#$ayD3wHoT{EUS2Qe zEq=ryLi%iMZ`b}tSYzHInTJ{mY{OXy0)T&Rly3ippqpTk%A{T+e?K}j zURM^%!ZIWxW$32?Z&q9)Rao;#KQuLv+^ft>o|6c@QD=_}ql%5Th=cR{P)_51Qxjh# zRJW<|qmpRn3(K1lMwU-ayxjsgKS`Q7J5m0kw|LQb=CbyahnoQTWY z?g8-#_J+=*r`Jc|A0(MOvTc0kT-tBLIIFCd6Y5iCr>cqubJu0`Ox+FkDWs^L{;0mc zxk-nf?rxh(N<1B;<;9PSrR4D<*5!DvA()O7{vl9sps3x_-Y_w>qC3OI!_Wyza8K|E zAvJvWYyu)(z*TK7e+Q#dFWd_7%;fn4Ex*lEY2$X%SP9K9d6yWC2M!3>3>tu}g4R*V zRMC!~oYyF#Izu$lGjfQ?q}KD$rpDMRjF?f>6kuBlE`z4Yxy(Y(Y+Dr#PKA}UsSWD? zm|ER_O==Y22{m%cO1jhu`8bQ05@MlII86NP>-_`<|Q4g1f7Jh*4%=yY_ zafIlUJ2zA?dT8&WTGLE&gvPl|<0zKa=DLzzPOU7i#nate!Z3u|9R6E(6FZ|(EZ%+b zsB!MEkGz1K*oXGdp^tGOWyF0SI{tq>^nbgX|L>uTert_v9gIv#Ma|5OTy0(c_qQUz z!2+;T+eysD^IV+aC=aX$FPzbq+lZ7Gsa%r9l;b5{L-%qurFp89kpztdmZa8Uo!Btl zu7_NZMXQ=6T6+OFOCou6Xc_6tf!t+bSBNk)mLTlQ5ftr247OV6Mc0v+;x&BNW0wvJ zjRR9TWG^(<$&{@;eSs-b796_N#nMB4$rfzYM1jb>Gu$tEpL8-n>zGXVye2xB-qpV z&IZjhW#ka?h8F{QJqaK&xT~T;$AcKQD$V>$$-$x~1&qfWks(mJ8#7v7m4zpWw(NS( z5j0d&Bs4g)>{7yzl-7Fw`07Sj6{vw5nwVyVt8`;Rg5bzISP26=y}0htlPKRa8CaG# z=gw7__ltw`BWvICf>5(LFDFzC7u-Ij7*OKwd7685%wb6a=QD1CjpQs$^2~cx`@xS` zNMz6?Q4OgIR8LYa&m`q*QJ%!CbD#=ha?38!M&7yLA1Wn}M{$nV3-G0@@bD#WjCYI) zKFZ`bf$tFF#}GYZ7MK2U4AKI-GY*y(&DCt~4F1!3!{>cK+7XAfKw<)Jv$b1vHkpC;gl=VNy?f-RI(r=&j z@Dy@&vHYi$GBI*-`1j-=qpI@{qwt%et&>`VuG+PYzF>DUM1!h|8sz~*0>sA7|IH_y zskL`MJ4Yw|Ru~}gzgCOOEDSyuM+ivsjt@13h-SLD|INP2zRO|RKEDz$_zlt)ZWYQg zKHk`_;gygz9b$7*)WKC(<}zQUY8M94a#Tu_OEyX$Lej=Cs`b}zjTYvv-Jt6E^_bV) zCt>gvm2{y2tK8Uy*;ruhTa_?lSIlV;r8b zX?jME!z32pO8`g9ga%`RQ*v=F0O`bnPZebx@b#ZfQWvqZPAb@zl>ORo<_o7Dp&F?6 zP(tBH@~c-Zfx?Ulkb{F`C1S8y3F;;)^MwWBiBPQ1D=;yC{M-i~ILSfh3K!Ai{5c?J zdLm0OmDsWuV>%}MT*Qf<$UT+M=7pMVdJGRi-rdW>7iM&2UO%v@>_!inA`JD)lrKC& z75Y)Lg~PVq0Ge}-g$8cy0w@sHjUuwMm1|~u6X!*fGG>%bAbv5cEU3nR6&6o03J2ff z)*M)kj|gyvZ6Md8Y!m#IuWuP0<9daW2gPDp*=aQA2qm)VLJ($UUQ>-4&3LX|)=-g5 zDTzngTm?JwMM46$Z22o7jlr3Vp3K15k^@=c7JJx9WQg*XbLRkdC zYapmoZr8J8X5n5}a2xjY35bC^@Ez{}9JA&aex@>JiMr#&GtJGn$)Tt=HVKx@B+w50tPaNkh{N0!^9>r<#h(fr3kP@a(N1!O)$rdf&Dd!hhJNtXD zIbx!f3YSHV50oNza38Kzd9Vze|NZlyBd{fKzZOSB7NqO*qDh)*>XW~VnmJ^ zji(MF3D>tHCk-^y37b-c7t1Zrt)VBlefNnY+NH0u=9IPbDZ1z8XbK{5_W?~aGs@o& zTbi2gdn~PB;M%^{Q*d9xWhw;xy?E}nCbBs0rn@{51pJ@6e=LQg2dvlq_FM0;Iel9= zz?V~4Y+a&wJIgvt5@%1FDtB9(A<-f!NpP^nl51v_hp$v8$w{ z=Rh2*Y?stNGlx7wbOLqrFbxg3lqpaaN{@9c)nNxe#D=Xouh@g7Wd}stZ!B8jrc4HPmOW%Xt^a!LcN8M4^efD8wWziBkha6&KggDq^9beRoiLH_z9 zGUiqkIvsoqX!3F)6qr+_HfB$D%@)T=XV3YUews|Tg-Hwn^wh3)q=N>FC*4nHJ+L$K zpR;I6Gt%?U%!6mxrP$mlEEiT&BVf$x(VJRuEIXdqtS+qfX^-@UKefF=?Q z(jc2Y2oyEyr3_bP|F%)C?~RzdfbNXgw%b_zaAs2QbA_QL+IyP^@l+{#{17?2dn80k zljl~W{3$~wO4E?SSij&`vnbpKCUzN%8GY^!-wNR8=XKiz>yng^Xj99@bTW|TDw5XGfDje2@E z*~-mJF8z}cI1eTpHlg*7?K(U5q3H%{y84gCiDbksT+HB=ca!YVTu zgPDuJzB@76rs{is=F^_95WD#mg}F*~wRr~vgN4^*Gy=hUUD_~f0QPh!&J7XP9zv&H zY}Zm4O#rej< zQmBNK_0>1jXd)Y3cJi(*1U|!mL(;nU#j_WV33)oK-!s$XS(mQqWqQ7&ZZ54iT5+r| zi|MH>VJs`1ZQr<{eTMqC#Y~41>Ga4BuQynUV!QuZeaFa6aP(B)SxC~V-r0K5 z5BJ<3nuAkX12%0k5qI=#D*PNg{NNjn>VUnvH!{DfD}FX=e%E5lw-IZgDqD$1an(zv z95TXS9wGg?Bl{w91nOC8HvvD1&ENr~L>4u{^bNaBD>ZHXIw1Ko!;wjz1%zZMbWE8# z7f5xlDTQWK%rH+)0KY&O>*EHs@Ha5t9ltEE{qv`K0tO?W=jgzciZhHZ4As;i<7{@M(!#&K$4UGQ?~d6rbu|rCYd`D!Bgha2*v# z?6){N62Wq7br9`S=y(rk$xKExQsyv0H~Z<~f!Z7~Wt6SlJBO4_KeNahC?2rxh%Z14 z{6vx|=@Pd?8vwjCEbf?V*zgc>36eg4u4w8WMluPe+qB=i60{qnN+XKmud{LfKvd^Rf{8@jDa#RaXtvGeC92KvnMDV3m2 z4Xt7QB96VazV=Z?RrMXb$#mb85@y7X+OE;c6PL94T|ssUhD|n8IM`GhqU%%}=6E(! z@O+LF*%Uy084M_#De*pBSU<)G3|%go1vt<|<(ZKk{3&*44f?ftxS-a(+@u_92o7ot zYq%I+Ztyt1x5RPt_1it>&+05XbK1B{-T~aA+FN6BiF@>|QCJ`#y*u z@e*p+J|+Jzl4qtDnLJPde6Gl8Qfu5eP#Lr_}cyBzGaR912ca0h5s# zbgocm38uvIstvyAPMEgVj^>{XqR&db7$(XJRTRiR@!lH>>CTe{+zRJEgcn{?M627> zsw6}Y)J+s3)u#g*Mo19)oWp785&T@;fee1**^o5#bgS4epuPWP>~Y2v-~{)-me7SK zd!AQUXsd{A=;C;8>vRTE5Dol&>XJ&AYMijyXV3|_46Fr#lz`uF9dT^PhX2e>lDN?r z>wx*9-Pr~siloVs7@`dn*kGmY0xP)2odnz6S437Hi&}MSb1iiwEiwfy=f;yg# zDZojIe7{n|lnmh@$rU>6-%oUGrG#^0y%z_Niq4LG38Yq&Dq<~B-3qLMHLbL;&A)i3w zq0}L%{J2P1a z2OC$%f4j5C`~!#oBU=IP{19v?%zqxLR77sUDKZWk1TEdClEz1yHB10F7>l{;9l0L|=ADc&?i zK#F90YE|)m(u4LGC%M^0?53NrH3M`xl2{P!5+fC(H)Yt|t=X~m+os4b6}Wj|nDvL8 z8n=Bhi`Mq$&2sm(8n4F2)~_ylMf-R2rn!V)Bfzhv7v2SF{79o}>ITpgUpe=zcRpds zp^3fse>q!&ohi{7gYJM|qD$1?s^vyP1XP=26O)1AFu)?|OCYHCJm*LP4*zJ8Raq1u z)9(U+oYRkni_C&!f4&%ORK?w$g6<;rT((@LunPCC_#2P zxJ&Q13mCI_U+H?IvV89Y)i_#NnNt!>xavHwF$|O zXuHG5oCo;G6F&W`KV4I0A-(zyjQ;ws!05mAr~eli{U77e_#bTiA4Hr~$mBnaBxQ^3 zlOJG&4aI|YIUi&Z#TBHjLS(GmY^z5R28NolKW$l^Ym#0I3|0lI-ggSR?CgqX8f;MBaPl&YzSG} z4(9gprQ%M^N3g+r;f^a0BNw0BQ9}e{Op$ssU!0cTdbP z1%BNUh*RkAe#+jya`#(*p*uQ|spESDMarSs8h3e`E#gtvYi=8d#ADvy9g>R@*^D~F z2t#h@kzA0JK)w;AMPg^lWi2XAU}jpiDF!akXK|rSi6}wmaK)KT*81I6M}f%l3XCMR z-&LC;?s53?Q?B;UuDeB{5^S+oOfSGE^CnkvgEc9^13~<4(iGap$VY8}3$6;-sL}t1 z4d0l&nxB@pZuYHH` z{ONm|SH}iy2^)Zg%Ou?*Q?I+u&ZmckE<;nVG0STB`M9GzLE5UAMeRQQJzJxXBBwA&_T6LHe4yGpP7i~lax~#Ub5BlJE zg>YF0Yn0Wcsv`EJIW^d7i>M?PO5_+)OxDS;9?zPfCH;#_rpR4-*9!|aogttErPHlR zUf2d~4Xa7AEaZSe)Mn9=Nd;=@JUDKUaJU-Rx~HXERZPZJTiBwHdXup>tP-Z$yw6H? z{D8e~w09((x@w&~)75oSpJ7o&u#DUKXAP}9afG;3qf=+XWeC!=Ip8PJvw~{@B3H)k zZr>U-w?x^Y3%$zAfoF_*V2Mlr?I=_C57F2k-rurm=_3`CHmW^yY`ye5aJG#E#oU&y z^R4vJ!2z7aF;V5BD1dbHn6(R25;-0cu1Cet+$J~Uw}=H_%79gf!-W2#1g=S`%zSN- zwVT1}5o>Hi-DpkU76(;YW&Y92O;@cEU^coXt>XfiRWI$}_*t&RQ_K?A8!$gpQKZe> z6VsBW458Q0>X1E#m*K&U%))^SmEntSPBAZb7VW{C@EA7Plo3r-`7EMb;;WeQn0bRTSxW7MTSYNoW=(qCsKsMVCbY?$#Z{|k#%NHM zA*6=sc(VKVE`UVqumIooHMGYRSh$SD{ErAy8%i_*n<=4ODdFErVql6WIx-X4fyaoz&jU+aYlbi=W`&5GJ~zS*@5IRv9cn<|il?|!d8>N94!OI0)aLF!Q0nlhtv zV$SFv61Ek9=p#mMT*~J{BfjK)?1ss~7B8LE@RPM6>=Q&sCt<9ZWOlek61x3T53zDy z_Ki;P_XP~dr)aCdrp;^Xx&4zy791bkXYcFE&ul#uoMVnctVZzl-Azp*+fw1N@S40^ zWBY6U4w+j|T8!q!)5)=7rk~;72u(J{qztk$Rb^WOCbU62Z^s|pn=)TqT4{gYcX?y1 z?|~>Cvir?R7Ga#&UI_thW{axhKZmGsOKK2*Z5|H*2nrEoD6q0cA?LAuQGqE#iVxT) zkKFW#vDut&E=}&^_xyn@nKhBk4S$!WNK~%$ z0c&2{SDdyuxlzV0ph!Peph$e2NH|n4;u};Z5-fDRQCkV`hd9~Qhw#l z5yeB&7zlX?y>QU?3e8P%Gzk1X934Q9LPIvcZi~Q>$tU#A^%^O!FsqRvO1M){#{wo# zBk9bs(!8G_zMYJ-^KkkOmXlld6&M}R+at4#TYfha^(?3_OqFsw=T6Gudap+sqFPF0 z*6D8MYBS6E;rkj8{7GbNPpnUPv9*l#u0T^M#yAbod>pw)srdC}u6;9n!}f|*m@!$~ z1aL-1&ei+i_Mkf0!?>5p@ss}z+(4GaIZ0Tu^mr{+M1{}bS8k3r~HKz!?C`p>TW)1H#Yg*vr z7Y{a{9Z}e1N<7QR%urOa_cLshyVKNaKNU@l7j~j>PeI7MIZZ|r0*YSjU6P_&ia|jH zDoChFYF-JCkoNDw*&*{QG3x+J%2L5_4`n1Tg9hatvloFoYL01#hFFj~!}MRSdgSSl z=m-yq{#uwWUIpuCs@%BEy5ob11|s~&TVX8~-XV)oMfeNdXD?Z9E10-tP#Krhiv$@dBpKj5J%t@Y2xI!*8s~Z z29}0zR`_9s&89Brq4Tru3F{G&uQu{ujBFqN`NY$Hb>qnXc(a!g%hbv!R@n6sNonM) zg649UVVIiIE)_J6eMZ?R^6HGdRMn-UD36*c8_Z2r&xc^Cs2p^v6x-_j{J)k91n!wt9I-~_PA$GNiLi=u7ixtk`YUQ4uIF+`SI~U z1J;MiD+DHLSA)nBsc8CJW1Z4F5uFXI0GzFHhs4egAoxF&>1&8*Nl_OA^!wW4GJCRO zwS%7>sOyj*5EN! zUpux=mBP|Q*_J!@%f6V&EZf{?`H}D&1^^@HO#Gta8P{W+FkdO5OW;fnD1|4&tlh3} z@YGnJ3d(Y0t#ep+bksNs#e?8*u-V=@#Dvz21#EB=jam5x3MtG&IuRHU$pr(K+Y-AX zn7FqKEk!?hw{HWBS~^ioY8Dbe(VtwFva+1h5$-}M9!~UYHGIL>zwFFN1`lcLe zwaMY%;tKHw`EL=C_^}jKY3YhWzg-&!anlG&@4E|`Vl}0q!EvCtT1I@}=Ug2;8OzB) zmllrTJ}RHtO2N@|-7)oaf*v0`{>2c|j?-t&WbDWOUDsBIUR24HnS0{I;>(%9+r)y* zg2K$nGPerx{E6HXH@h?eRQC~Y44A2^$`xKRwnOj_7pT5_!?K%>JT+F+ z6(@ZUF%FqvCBG2v8WL04A5>D=m|;&N?Hzcdj=|%{4JK2j_;hMKOfU}I+5PVH87xo# zc>v2%1gFE>V^6x3$7#ymLM62}*)(ex+`ImB7=eUwa2O&zcN_th9iPz)#fXNbq_VnK zg>+Fagfb53(>-Y^v23^|gST@kT%3pG*YUyrd-zn|F0Cr_;Qh)MO;mTE$%x&%B^Oc= zO-<|3$Nplt0sdxXQO`|RVIbVxm_^24G_6XuTxk&{Yyl+?OeXa-!t}8&fuTGLZpS|{?$S9qu^8TDrgtdOu`4*Sqx20lCJ(;z6u7&0EbrB@495}e zvjfw8yG7#Eo7QX+`k$3*tbTCwGm9LGOvTam&Kk&4&(T!!b0d-h(+s160p@Pn+_M|) zwasiA7r)El>t5DJfiBLb@2=gQDN0N*FfYuh&F<6BNcc)=oqju*S(+ucbzy4pyN1%s zgS@}T`xoCKJdeoM>hW-Zt9xSNRYI8RfX^{UPSJ}y8$_k~4-2G8KZDJQl``0lf>>)j z^q^y@`VIX~W%W-QAF*8U#?c|>tGQ{a09;)CL{-NfEv_2<$o(R8`V7xFRTl$)d~KX! zxG^v#xd(Z9R*`P* z8NwYSrl;qaYDzF0iB%{|A(v0($}TDr##;!y6paThkw{fnuKExakKusCdM>46hESJo z6Z4inrJpt`IzSB{l1R?`XS)o3@M9OZsiP&{y4g5QBH!U*Fvdd|9inn^a}Nz>2&)`? zh!|tcpGBMA4e|H2Y3)~7iyNUBsc|aN0$HM9Uc2MDIL(61;J!I)NmIwv>&&25`&+6M zq1}!I%Azc>=L(6nYlCWwU59Ea*szPa>sE|5)2pJsAnOmce3ZqxF(4^b@uZ6D1K#-5 zD6|eu@+l+j4}V7yxluQ@oX?sla^=5dw}yP&j6E+69hswg1L1c=)OyvZ7^wHQJl;ml z_2lX#$i;=Fs}vkh=ukc4y2Vj2Lu7vAHQ*E%@5?3`^a{BzDVU zF)O4|`;uuAO@)kfdwp~fqS#rR$4Oj@c*zBS`-fL6qu8<7qzl8rl--^kjiCV!(vbxC2vIdMo2I^X@+ID zcT&$52_`~JOBXh&mXX+ceO*m*0_=9ArqG>xjMR;+M=q{e-N#QEj-BCAzAVeGSrXNh zCV`uX4qS?7l$u+*J~5P?9xlU2%6rgo30lJ)cd|FHtEmloD@8tO@5y7N5t*NZN|hrm z*0FP5k0_1u5$>dp#I>8az>my1NoIAqBZ!Lx(!ohP^U@&Vmqd8 zH=75V+`}JpR;Wj8!j6BT1WSjMs>H+3_*52JYs(04P<@$3WEVZ7V%N-CLN$onNB~*- za-hT{!s~K{EUyaw7zDbp7n5T~SRV3$*>Zhpg-*51L=Zj|oeHx)1Mr4juj_5;_<5%8 ziMWWR&MhgdLq0$}U0q=ol1xb)TQBdcV!(3$iF4x~ue+F-gFAGMn^|`*YBjuP=jx!~ z06>UuQAq?Ix&zn0^To|<4!CSXZW7o6VrM}5dYxV+Q~8-h^Y9DzNs{5%+kyFy5cysy za}2EkZyRxQ^Rgq)T6r=({uw7y@%D4S?wd{Ck@D0(;mjg4NbY$Z$xd6rCGrNITO04Y zO%6aZ!9hMp%kU=V6dLc($d`AHMbf`&G9BXY%xr$$hovCbBj@|K2-4_HjW4Xn{knIL zaKV)PQkC?JIKYK?u)1`rzd)G(eO222!%q#U6QaT;SUl*MO9AvJ_$WC-@uTOjb58L_ zQo63V8+G)0D~=S&a%3>qqG`7N+Wfi$Logc=SXGBq3&TV|=!!;Nzi4VeqP9=hV>H5k ziX8p2v_i>9nc1rQm(7T8t#sTSGnI9T#Ms(_k_%sm3mT6gc=YrdUm@Ip6xRqL0H93*Yx0O!3Qw+_Y!81*n-ovS%iBlXx62TFNbk8K-j=LOV=1s zwc7i_TsS%sk!R7r81r4v*Ec`Rrl_m zr2$@wBrDGJ1`%wG6Ar259e%+MkZzK88-X>M^WgfA@HcWJmPUeFdO?d0>gvCTn0-ZWgb;$}~gdQiffS0?*jk$T`izb=V-&N#O_U4yp?Y!Mdlk09!o82t}+5dEvSj%vN5 zCBperFlf(sXr6C$n?zYvm=YYyz=~W1tkhvu1wODh>tKoBEiRB9*Py%96luTxm11-k?Q=g$c>y=q9%J< zVbw|kc=&DAiz8G*&G@8XlevEthbWV6a7nM1@VjKNkP|sl%x3(c9h#|9HIdVuC_??C z!MaVTrRI4=oMEugDa}D)#f1zPsr&vLR0Zy!7;QA4?x1w?=X%tH7o_(2z@8LjA`t^# zft3pe@**E=P;MFXEB+)Zh$?+;5%i6ECfT?A^~N`o&QHR5@V8a13HuA~omH+0(xm&s zJn#ru(@aCcl%uY66t2-NPi-*^o`hAyJ}I5kdqib+qh*CNP|jg>f!Wj#HJ<4r?4uCX zvkf`dDbhurH>#bk@3|Ap%0+kV-0PkcrZb0Q6)EJKBfaiae*!zLC7wkQ?cY#avSAHH z-b1`V^N9SgFL7-JrVQZS2rsHMA5v)j^@ga==T4XfE9yy6w7~pXILh8O)Le{Zg)9`|o`-$nca zc~hvlgOB$pGXop$oW3PzOuUbE^uRf@bo%^%%GEHQ}3uc0E<9SxbN+Fk6DEin>4 zHcD4f(K{ENOe$J0HJ#urqwE!{iYCcrgQT6kUmRQ&pZsx(U*x5m938GK3cceA-25P7 z?4_>Rtm;@LOJc>-Es0d2lZed7(#_R8eGm|eZ(xhjbvF{TQvs1jaS#K%R>_hqN0n}TZ* zkc089?X9=$pO*FdJ8a~1LwKU&Tl*+PUpFFBdK=aX&m5jxjDg5G1pXXNL&FXtQoDIi z%I2VE+_J15PN$4XB^X2Yje8=^qT3Q6Up)7auJ|SXIn8t2lJM#_5ql$SZ|nXfb&U<5 z+WD;cxsrkAy@tew0gl8PHWX0(qf>97u#=sJz7BD=`gp*W%GmlPa|+rCER@9rjcWg_ zl26OYrAyJyc>(x*jhp9DekXff;UF2NN;Ui}MJ?5ICzv@f9ALbJ?E#ZUr9Ic3 zzA*o$&I=Ta@JfZOEAMmeNUz9k93p!8X=>FBD$#aW*rJBSOJG_{E4u;M3A)vn3ZA*FCGn+Fg(4w7}cEUuvHYjNe3srT? zjGbTt%LY~=@?&|zrxYJ%v<6_xj4<+!VwleU+BF+z4)}b&?KFik zy?KZ%qJSTxm)WSC(-)vC z_LTIFihr!^y%i5PBEEPCOyW1(0O<=Ad}++TAQlUVUet+p^E3c}!Hm6Ker0kttjBIWHFAYVE28@r68QPb>)Vg<;d0ndg zIOg|&%Z^&B5koUj%;;F55>#Cd>y`X1^41GHDSIjVmR%4uBt$XKaBh6+p3un1m6DKK zM5nC$KuQFHa!O+A!tnBN$&WmSvCPz#nQaEXC!g(?sW+Y@AB1kdg2dM^(Gjmzs6*J zi>IYc&r4tXJ{{+;xx*UGux7GmUyf}GKo{&yc+i^CQk+fM5xwnR=XN< z!u~>Gl{|8NtTsKC_us}+!JbSFv?wd*)?I^VPt2vT`c;a6orPS2Qhe`>N1KB~dB}yP zspLQzZ>`?Hbq-7qJC#l@Vh{gOd0-=i*!QkM8LpL1X8-}g1mS#mh6v^#lwH+V0EAht zLRoZn@;eAS)m=80s0Jn#+sLq@zuIq|XFXByZxLIoN4=#LqQuVVkJJJoqdv}YdIi8` za&=Ppx)n$aP&MKW_^PY6l=m-iPXIGakyd*1%=})EsxHySwRk^AE?qcrR8hTjF`nFh z)+UT>wL0VXkVCY=24X|7B}!a=Gf)c2+1jXZ;lwogP%J5l_LHb4lWDj;(dv}Vr1IJ% zBzmFhafX~i#<1bqv&puIYKuHOPY|K%X&v{<{=yTL{$8uDcy(HHi}VDVjHC}Z7W0`b zEvA9p60jBWkkB5Rk#%5BJPS(P7jy(H&ZM=!PzvrzF1=cb@j0B{!WqXMl>4hvAUG#n zJd@sf-hvm66(tgSb~I9O>_*OH9ggr<9(jkPzpUP5U;9oi{-`RXFkT6&7UzshGl7YK z=w!GA{fajfE6<@$!92K|Md|hQp!i-X2J~nt=D;7#M2;}9l3LG<6`3C2w+L(}Swn*C-B*?`-k7j87(HI0e zOg>|2NSSo0G$Db|yJ=}l3XfUHc3P)1NIM4OhMgn9utTLY8mQE#BnS7N{&WXwxbPTC zj>^Vmu=6JO$5zNwB5NNSl0w;}jb@J-VA6wNi{X~PSBBYYx)&mpWiwGyMd~%>340*O<^m+;13xv+nsl@@4vWer8?fJpf?QLDsIAYG$AW; zLaEVbXdlU68j5l)of@<#27i#8e9acN)RqV5SD02bMKnOYW!RB{72(fvCCTBSVi?ru zbgDA#*GRW68N(c0E>5u>u(SP<+gV#x)7`Bp@SBKiVu<5JAQnY_TkLETuOirHXdSvS zvj3FIepQF6dAlF4aI!UHW_6)6yAM7CrBvn^#Qb^(|KMPUas1SycQijlWVnLIlvayxabGnXVuaQ^dHa@y9)=$QZH>SPegN=OO*~ zE)SFDbmX`%K>u)QKvO4)0Q6_1yp?lfgooarhtt<$z~YTO+(JVl(~ASc`owLsRkis`U_?MIJW!nR@Mo{TY+o9Pv7gjq0Br6 z69CC^k3Y>byZiTYSu$_l7lJPB2#srl$j1$McL;9;1JwOOnTj&h4}mWH-Vn?pBA#s3 zjm-omv~5W85u0g%GVKXOn)WQaVM*sXOrslhX;tKH6?3k};k`m#5;f?oYG{A|jfzVI zEawoElA5$S+%=j>B{ljl6OB6dMOtiz$z|zws<7A7tg64qMADNf&^>0E_v(v4Xo_qH zV^U-nQmvG1&4lmI`ITySApjtTHJlbWG-M3T*jAxeFp8eXd~QuT_;Rtxq6gbbb-=tw zoQ(PY91W&wSS2@?%S!N+c&XI*-Qe>8h;>EoRGL|8iL5JVmPFo`8mCcY@G7$%vVy7X z7@ReiXO;L?;tk6Mm3?VrP%a+9@9N45(_m|XD$^pZCLI=|=N&b3Eye{UTf~qseLt&P z!#sl$Vu>mfVC$4UM*S1iA&A8WT0&j2yWtx^d_y<4cNyNemon|ChjXI5IDRb_6+)L6 zHL>y7N+Zt&p4YiL#W9q4j^;U#_Uo|iALm532s#R|g|RtF1ga%u9(|3q*VEV07-Y_# z={jfTg|b)%84CRox5B4Px#rve>wV`e>F+Ihvw2o<_Q-Nv6Oskz6Xf0(P5Qe*HQ7l- zcH%D^p0}1DkU?Oh5Luxsh!wO zKUM!6-)%F>W(*eN%I<=x(m0rDftloG$@?ufi_0FJPvZ3#aSQ)qBP??BlZ)n3kR!u( ztnUxe)+T0*JsBGnx*NQaQ*rbN@u7$&a*QhLA>#~Ru<77+YbIJviqYiex1fq>1{FT# zFdi=DsQwOIHD+foydCEv&;U6m{f)}zJS3hga=b91my!N=YxAFN>}t3rbzl6j(22F3 zN=wsJ^$u!O$eS~g%{1`E%Z4(MfN(74t3fvCmpBFL^Zwb}W|;;%1`>f&|3*$y)Z>cJ zb4L4u3{QiD>q8`;X78t!poKbPNQ3F!N5@gjzIaM@VHUUjjLWq@kvi9sqbqS?nXGE8 z#+GiOoSb3agPl)kT>OYk63q+oSkS>R1&~Kn8mWrR@Ghg2kK(O=B0gr7cqQS&ZU#=n z!fuWk@yB<^!ZQXKgv|$6V&t7P%_Pw;Z6eX>n7u0VO2tT?Md1A_{XTzc4f!^fy@J`@ zL_xHu4pQ2%+0gi2MYpK?iQ^gAY+ZY~Gl4zpRA+4JCqhte=){_!sS#6~-(u2O33{G&qyu-3N|Q&_I& zrYu8ewgXs?(VGq;pSXyDqUfrqm8MV7=*kn-gajV?A&2rCKCU2b%V#8DjIS?*Vby zKbhSHwl(aey@M#B8n8X&2S?C9fc+T=k|2m>1p1jE^8a*p7GPC1+y5t}yFEv0biZjerCkVf)}=vc*AQeLaes5@b#F77Z6qAz%l-99zN7!krPb@WE@*haV*6;&%ac`t z$p+!J!?T5Q(0fA5a}OU8+PZ!Ndhf30kT((m^9FiJ79WS^vcFZ6gGuSj{S`e2Q%u8$ z*$=`FNUwnT3MQXg2wm@iypIy_wtTRvyLm345nt~Hjh{W&yk9bNXi)x$TYOmqRkBjR z62UrkX=#b5CsQ=dI{nd9hLOmmydWim_?39xb1J`JjsCP(>wNM~^8+bwt(VJK^`0=s z%97EYPT=bjs((ZFX-|N_y>DS zvWRyIuDcghz}MpyZE#*nQw|a4uW0zgqtA>*CLBdpjUhRD`mJFRa&;l=cRkT3S(l<+ zO8=_HSCLh~y|ftK(ajUECd|EE=Wy?Hb%c%#nHYPZLw9akcR7u!w5#-PioD>8RhE)< zt{&UjCzWN|o#^vd8j;6KXf=4}kMkCW| zVSxvE=u0vh*r$0-S(9P7Q5CW%^7bKVu=| zk>ZOJ}2*@xw z%?i%k;pi|RUQ44_+hrd+)y{B|7lfBZp}F!E)I)8)h6ld30f2zQD zTA+dMr02cDX+vCzfK9iwIK=x(6Jyzg^uR7;c;;@nWi3y`O@AqwhJ>;X- zN7gfZGgG5gwbGh~E(12E`qln~DWZnEFRDh%yxmP)2=<8>_4(`U0+5>T-4EU{^0T?< z`+eP>KTJFH+2mikxF_l^Z@%c<4BZl2RS?NPZ1r~7eLM)%xk}0y=Acd)Cm(z~Xvwb0 zQk7zx^wnc%U@M7vM_a$zg(1pPLqISuKU(`;+GHB;XjQ`ED5yW)tP!0z#M2FKs+Ds` z@d($Yzm}Bw#6VTT%Ge5*n?cNZ-1wB^I44Q442Ll-=xb?uqN`n``RUrAJG2xmJW}#I zW1SCEJv%R%*ur!4a{!F-lTBUWI$4=GO;;xgrKZ*Jp3sa<>ilJ{rnNT~(~B#*XEmiU z1~Ed`QBgYpk>YsHbLx#%E)o9--i+ZC9f^_7T3q*re!~_iq1d4WhP8%?V(#=QM(g^7 z>2+F74STNRx~BuypUTi!+)M{gS@jyMH($ZDu zKjsY7wy_tY=^3B$W08}!&<@2c!l~K6&#D)VB-K$kGlCyqCHZOrNP@szFIP8$SAP6l zAIjazY5FRXfEyma)Kg?SYc6gqIrvj&$otnW`!RzBpQi4fq)s=P5CdQP@)yndY7bUH zan{vp_Qu7}wY$KTn$j1%Y@h6=n?MZNqDJhm%WboRANR6CQby3{gRzTJfUkwKimRra z>v20v{=}dJ`%D)e01bVn*OnnAnvxkDMidvnnJEF&DTbM&P+`Ujq+6c9syhcdm!joG z*1W2nVX)Y4=7jc_kF3u24hP6*6e_ugdd-Zx2G;^;ugxy^C3B;tZE{9i)S#}n+Tm^Wl z^%KpO#g^>$))G%Ak1-6LUD#ZTRTn(7!9<4(>I$Q9zeW_j9T{_T6J6i{a*yI=rhgd@ z)gG{9+1{|l$zFGeY|`t&%G=$#LakN(kclKjR)UF-Ix%+c&+>+~j$d4Qmb}LruYMO@ z`qpSxlDi`75!wy{eqU`gG<%ZOL3iz#AK@!h!=>|j1B+Oe$GKu9eUZ!k_(1T+S7_kA zbJn;fO_sAts`Puo#$t6E;ze2?q_a>$w#+0nuk}*bYY8_IQmYk^aF^PtEnm9%vS?g- zl=f(*i$v;};DFLu)Ie}{;wBfYcRZ;#gqu}?q$J)G2lLswTD<(sxB!k1pp9in$Y8=k z^3JyAcETT9MmAB~bYMX>W~mpKeS-AdzQ{3eH)NL0Fva9G(r77Eq^5@T^jqfFHlZW6 zX`)orA@BS6J(?KBp+#ABTs)dY-6)A)m=B$=fl;)gp0w5h=kVgFEy%>zT==t#)Oswq zTr?{tmWGWFbDOksn&?;8ZO@~z1|4maoHqnx;)hZai1Oa97qKZ2`=>=Tqbi7E&k^Na zZ{=(CC~B6eo5t-^lBcfd9J7-)zKvBA>K}~;QMU(%+w1B)Tm0HTIfLh#lU;3Yn~+}d zUP0S|jo8kZ7+vu!d=$BZlVeRdZn#XTYejHx3KQ;O9%HU#dW(r^FcXBZC(y~Sm~%N} z2AJNk$S5a5XzSgPM7Rj`gO_&{#IQ+BaJI7%Cg(lRcrdBsB{DM zT8d*WSa9l7$|3s+xddzetVv2FvHpTmi>HO0ST5olCxQvl(GCf3Q9y&j7i|TuS52RC z$Mq$-RNqf4At8+FuTKP}#H=tDX#`r?5dsa5dEA@$R5+ZaAl)jTIpWtmtDot`nN#*n zhU~NvwXJ2@?Ng4=Ga)ngqKekQp9>riEd9DzgA}4BUwqIm0%Wss9jHUl$nKYqO;2N7 zknpSn9IQrcJR>i>8i4TbCiE{yOjELbLUDeF)~y3Xq^W(@CXkZSMd`R;HHADm=DLkJ zS;1I$?g$Acj(p>KT3D?`z_4LUo}Uvij?k=_H9S~+>bx^)AG{@fB`}K$xi6WJ!FPJGW zB~LoXg!SC`+S#|tF_WQeoMF^8u?W?f)9v=3VwpXM#@dD`br&6k3%WzaC(pjfR0`fM zChRRAn~rhB-s|T5e1XI1$7!j+-kyB4Yw?uPR@@9KfpTk%nATjRS13yeX_R>U?NRR* zYr(<$9=%ADVmjc*1V?@FRwNrtIjAjb6~xw zC-sWFLtc2tkj`HGvT-)9R$lY{zLj=HPa%BG;Eej@!{!SgZ7uQSkiTpuyam5P z5rGi-YQWO|GMX=FapkU`5NRBgpyZCbC47f9)TZ5%PIz1ivCfeoh~;Vbi@p|Pw7gM> zwb+um?aH84>hd{#m`B&9Hw?kAeS3;L=R7r;t*zfqC&7JCTJ}UUynqaE9fG)Oeo+9~ z<)#K&_ox+Nw&lB+9i|2E!p?w#If|`6#-*70{+ZT9cyNps75*mHJhbjb(M$RiL#Im7 zkt@=c&>5xhMt!=^u@mJ>AD$D_6u+1VyRkNNNm4B-5;&h9$MT0M8s71AN$h*tvfb!k&(H`x-=+RpQI>om@b>eBy%{M}3KN2#u_7ZsoV&Xy#uDxoRl2 zhZ9oKR?*q};PbY(m7gWgt{z{7YV^%w zc`Y^X^W2*`zFzR@pZ`FAYXD7ajJxrE>}I9XGO?tURZlH3Izhh)mjN#;L|i9=q<*Nz zeJ$l3es%o;Vkm2YSg0p_sEJfD;4905eJ~)3KL*>sr?_0fwyGKtmV*Mx?gOY(=^nPy z75*rmkv2($3TAtHYhv>G)jB4hBOwj?+DEI7B7nKguhhz2Yd1 z5R{LN%C|hj+rB0#%?eMKUp2KkGARiM^w%6HC3B_ajcD)SC*>BKm^LzSenJ0Ao&OwF zP*SjP9n;qLfKIW#zSsN6#KjQ=N9BF<<&EVWEqo{0Wy95oba_&mA2}DQZ?GFIAE4+$ zTSWyjBPuJ{I>+2{`XjGQUK|-8z?*tIei@>sC0eceal?yJ)H4CGLcpm&tzj$W8yN`# zWW`Z58t<@KB$*M=mUB3S1Ewuu;KvZt)Q44I^sc9(<6KD zz8jzDcL^6W2q>?&+~@GAhGm!bSVyKo4FcZIG@w+Qpt=z*Ug35;iTEV_r3KuuIY@AP z86i%AyiC(GJ?msLDzV2q&uEWf<036blx`(bK34rhL@TD$CD~KAPmc@j?tv4i(U$`9 zcWk#E6!Y?LEsmMJ0&nlU1XdZxd)a(3uMfNLXuUp;?^_>tzV(jaTa$0?-?6+ps6I8M z^B+WMTXsb|tcon?N_dCOn5B9n=!X7x%?0 zTWoPArre~5nAqwvGIZK;G@h1ctA0q9aR>+@?}8?$AnXuMICs=!+GRwXA9E?Tb*cs~c2&|aJbq|eJ7f#q| zoxW$gW$NCNCCs5dI)Z^%IkU1tA%66_qyJRWe0$h5=C+eor|YD9VtX=mo9i~)qd6;iM;BM3`Er9%Vbh*xkQP$9s^g?<6<&loxpnjh84ZhlM9LxMJBc zLXJ0K3!L}(&LVO@gM{JDV-#1QVN~`dv!T2 z2Qn;Li&$}sd(ekuw=gm4*!C?zfH%!{5U? zO_#Y7qV!K-j*(lr3xK97+d&CUgC{~Jh<6M)O$r&FwN{1 z20nbi=4jRBh^n!*wjSy8azByNjBI_hrIYM>2DjX@lKe#Cjb~HNQHwH_8rD&4I!0l; z_yD1aD4HlIRpaTe{;-Dp(o62$P92GK;Vp2_eF?x?niw86wX|gzR^&6S9>(;XlZu!P zg%R|xezBab&$a_p^tvy_W@JtUC?XN}cgE^{$r@Jj0O-eGw1y~*_g%tgOnARkghNuL z-{~{vK;QbpL8{T(kM6bO^)h}ux~es@-LTd;R=9)sxy<}5O;v>vrHj%91Z$l;<`Y(w zbdlOcHl_DeY2!3@#q;ILT9*;B7%PjE-TI@nj;lVk>o~L@x38XcbQ>sb4Q_ergjle2 z=1TP)RfEaI9>j4(%Pj#eMlOU;E^SAsx1HlY$8Ha+YL5x9-9of5SP~`Q!TTkHjuEe( z^@Be9fgW2rMRKH_{6?-ncAL`peXi#-uUai?&<79D<|qcq#{*VhfR0^Bu#$m}waU-a zf?oVYeZ&@3KR+@Wsj@7H(vYJuPF8)?g;g1qgAbPp;Ih|4hUftITYkRimR-QPGaWd7JcGhKSRpMGT&ZPF3KZi+UYK+VsaLymr zv>(Eeqzvw$N+M$wu# z>3e49=_k#bazg|41_rGVT0nT<(dcOP7(s1Ur0>eqr0e92dZHT8*{A<=?8f_)wMpo0 z{|aanXhtrN0z4$6y^uuRVHQ*`pV$MvaOW$EvoxJGG@+{pg z{B(^TDMUY~v>>L4)O#sr#wBegOIOE&*2iEbQW`BhEFF0u>@prRi!1xGtL|1g#KAS$ z2z`cSn6L;ja0_%*HV*2mK3AE;kjTw^YqTooD;21_$*D_&YbZt7kr0YIgDiIM+h3av zgXsG{{f0}-p6NrnC_K3|jZ}V2#|Q~}&q&yQGGhGuzGQpOxN92O13je4X(I|k==cr~ z){SHv(u91WcbB0wZRt+%i7bMlv;!;=?yyQRrb<4vGj{OKNm9nxng!4NsvZZwIjObb z@KC~nsdPY69@6BqZ5_xo2)t2U7f?&S-~;ZL?M-P+2NvUqJyv1rd0k&{^ggm|X#DvU zA1-EY8=0$XfC4GdfipYcF7$esav-K`gw%(SpA#*Orbj6niv@8kHC8^~J1)}`9(X#r zWe+dN@#5LahIxdUkkOvtdVCuX)hsK*ev-=yc~?~I&5QnUdA&FOi2aQH#JHqpMANea zI;p)iNmoZdlH(Y%N7`Q z$tJQ{7&y_+s7g)E&Jh({721M{ps2~O(9SBcraCmcZ0}dc5$rEJ!v9Pbl&6ubxH@S& ztYob|2_`2;c^Oa>H*AXv!H4p7jIMDi7;0~m>)a$fmh^tqSUKkGutJV0J%@winXVE} z1%Efz)uZZ}4@jH2eb^k(9K)`8{RrURx2bPm4BcAoetOQG1Yd9lGtN|#HSUjX16N>h zgp&z_RHqL2#CB%Ab+D{k$HbPfS>)o3Tge}(!1u2$?BrpEgXExq>_cGo??dcNzwR(V z`2az=)m9(}T9VsMQ)TcvTmoO*co=y?Ehmv68vM8`XAYc}We zjk&~={oCs$W&`ksP}g8;6e0#Qzfi1(I;sI<8?wAN#=S{q>b48Z8FtBqMe3Lo?t!EY z^itX@b~44Vwu5KIb~f1^NSYKTZoKLnZZe6uiSTR9JbuYG=>r+hd$|$O8?Z9?6eW!k zTvcHux%(;faiU}^r84lESQ4bMI=%MtQE>xOs(mCe>RrTGIvDfQnE0D5LQjK%wz@pq z{80dAMVzvl{BgUGwK)lIPb$1`LijJNSCwa+)WkhJcWqqlj9V`-C$fYU5EheRA zYafq_r_hB0^C}Z2UoB0XSs!8%AUq)yVUO) zwX6RI_&)zfJ?O}QN})B zszeLFN+26+QHH@RthaWS#8B>Gj$1KjY3qnj(efg95O48)}Hn;x28!H&jZ`_1+LeOo1{$L zw1a-o%V@mzgD3f2q79xeeEC1aKOyC7B61gS*S?_Zh`&^p>&?}@RO{q0!(DW^ec6;M zYT#36iu`t^u4YK394UnkPHrG6(vS#2#W7^a)DseTl(SK{_mRx$SSO(;R_bGn<;tZ{ z)`77$`ig8YMyqtHF!Oe^VW=Tk_L10)5Fg6Lmp5r4<(4)Vuimrx8er5B(n2pC(7r5? z#p<4o`2yc+!ZWADaFv&@35Yi_ve!%T@*JOz%$|SD0Vg&dWx_ie8OD<1#3l8(_F|Jo zCmXF1Uv%5xfF-Fk3?4k)4sbvl&!T!idJn0sbY#s!A+COh21I8hGu6fXK(MHhwc<^7 zjk#}tUy&wBpV8PzVY|f#+K#Y!YbCTm*g~AP zgs!E>RURoH8CYZ1E6;(H%K|7or+2N9^-bbqr-9b9nv)Xdd--LXSApu89O>+r&{j(e zsoCK3=YM5>U@;s1%m%t8n8Ez6Tl$-szkla^0A(mQvov>gGWtbU4d3`(1<+GX_por* zJEnKK!ZAfXWakj?oanK>w98Y9u$CH^O}GD3ny%d#s%lo*wAAtBn7P_V4@?f6B`EFdP27|nUbv{J6fxz z&di#|ozz#*%c7NKR-|Rr$zJ`G^W7UZb$KrG$#u0iQ!4Pom1;dBDrR`K5>p%fuIim| z)uO7-JkL@}EF$p2sMc%(@TkgyPCk7K`eakofj`y_h6>Tv{FFOv?|n8K1nWY~c$J7O zo$OnJ8VwVPt8`m#*V2+6*PL2&p-b36MazIZ^`hSGmUdct9ltF~lGm8yY_CPrcVPqF zbm=0sw{Pc%=v4NPkOWx#dk#Lxd4?Z0s9pr?U_k))RlmZg8}zO3szcme$P5m32;ToK?74f|_(j%4_CBhdvdOZ zAAS*wBz1AnzmDxfU@^OsTn#5a;%Jrku_al3e{

1bvi{DS7E@q1{$_8->K{_OWv2 zCZTgG2Pr3n8|ec9kIu&uC|d?k4-cQ4#}Z`qDX5Y2mhC(jR1Ms;UG4Ho$DE|+SeJ@{ zJQQhAXj|<)*t3KiOWTuh{Wd^mS{u{&ERV)OpZwiQ%#1->r9p zSK_^*U~=?ywH~4IUxb}{0J!SmL!z2Tzq_PpetoC^_az1JFg0=gMcQADuOP%3=H1hH zH_=dG(PD;d*037Ov5G1924U#Zns?~fs+eh1%-bWqa%ssm3=nio1r3J<4G0IBETtr? zycs~0JIOn;MecYG=~OQsYHIrf?~A5>_ob%8+uOrVA+VCJw}{lygrBBdY1k<8B^wf6 zl|<%N$7)fOZX$%y>4ueco_Gb1H@B%XrKVwrn6hUOecnc^PU0rFuCB5=*2;|u-`o(@ zL*tr4bnQzXYLc4XqFbv5sK0}A)`}`8iM8ehtj#Oc5DrE;0VxbPmL@BUa_BQwa$EW~sU#-LP0?sGmqfUGhGWcciGZ*4(}u3z=@b>Ow9DQe7lcO3K}BG3j(t& zH10>sK!&4Q5-=gN@Nxj6{|*nuyqw7KZJ1?p)NUJ?U0bOigGdsOk}Iz&9PmN_5=W*Z9M zy^pA`&dX0oo6?CSuhE~(pYbLuTPp1a1Fa@e3Lu&mmgd$;D}&g-i=D-{sv?J9kIr9r zrX&Z)aFGK^kNY{LxrotP0}k*;uN12i_2a_JJhKwh zBt{D-JRxC$8U+-`u1xD>gJ^H4lbW;7spI-=H506i=ncdK;xq*L6f7jVz$XGMg5aQk zHRJY&$@g}i_SP##iC?lR?ltnWUTT-UDlq(*BTQaYNkg zNG#sNoo{WmP+Vl}U~?+T?g25b$E-7iwhu=VVgw3JdFXm~ba+LC4p>CP3~rNTiNBl7 zL{RfLLepNPEtZj}yL_#R{(^MqIlG)c0Va}>U|9Pl&B_3tV;Ps{r)WqBznD7FcTlP4 z`JQe2DvGhmeeHGGX39zGyOOxZ3tq~Dft(BQ;mDXwwJi?sBtxo$Gf1SS2w*eQ0p&RVMNVi@d zY8v4J0(n}%6*Rw(g~l@sUuxpiJ*Y}7TzBQyU+>-qWm*InUeGt@)T9g^0J#z4){Lw* zT;69if~U9DXBR9fgVPlYy7aDhJU)gDC?_GHQtwa6QXNaah7-CzA|Fx-lH7d@N9>38 zX(F&fd3w7AkZ+ha8-gKfX%@_~<#HDs?kBg5zW>V3%Xw5jwPs6uni{7r zd`EfPYrA*SU;xDtm@E>5TrJKlg5o=h;NSXk)pt4K)GbpP0xkUg>2o|oG=`UnX7^Un zb&@8d6Fj1cBWW^c(K#Csc8xEBa4KfHY>8Lp^77-lhzgWr9kR9_p+g|-9r?VSv?qA%^1O;cqgke)%AqHlR$B{!Y1Mq zj|)Ecg?{_!>kGDAwGa7%cwSUb{BcayJihkv$}ql+yu=O}jVvAFdC{Hjh$4}u+$mx% z5V$sUiGCX%D3A>bKwY8HR)Gv*lisI4q^3vJ*nDwj|mtr!0r!~+Qoe2cw^jPCXkT7tI*01|w@ z&gPC`?O1w7hQ%=&bcHi7(fqhY3${~JepA7y@^aLwHpew^Yk$;R4v{ASHjXjXtaTc_ zuz5*nXB&PrcyWx#gQ%?HyxawmS+Wu(7ssvB1UMh!1$to&o(mv_f=9~!9@VsJCGxpu z`>g5Sp=xDhpsiCy^y>=fI0DON$&pb7o7^d{@@&hj3!6PUd=vA;G;#7&8ChamsE{`^ zY8pDra8Jntp62Ivi)Y`*XbpM60s06v@Rz^-g)TW_F@B!~y7!4AJ>37mAuz!(!C+xQ zSR61?u!{N|qHWOeR%$RXRL~vpN0SGri7-klNHEJuivbi=0qSbdV4&ghf4i|7?$>z( zI{qH?i}`~a7GyB6|8pZRq982+P*r1+m-t&(%U5#ZWFQd-(CXKLHeN@y(c z;wqq1hzE@q1b$GG0VQ_)`{MeylBlVfy%UHR=;Z98>T3M&;{0i?+0T-Bck?I)AUQrz zeF**_iGu$JlCpLnFv`D9?q6R51jKPM{Rd6!0FF#KP=O|b3iQX*TqXSjO?gXaXAmLr zU#g&%@+XpjVArlGkfaPKk^PUSnMLsjlK<9nH*zxl^V2-jGC$4+HGE%?F3%4|y9>HN z|FJgz*HW$VwU8$RNtuBf(2vdZhW3x;R6%eoJM(|2zvKebxCh$s5J-*fhZ75B_yeUs zFTrToFiB^SNH?gV2>l?G&h!UD>UP%uKh1L;Er59!q&NoZRe$VEf?5Ar^&iUad&2gQ z&WE`E%lTg=_3XQT@gJOjkAi-Hbbqrl{(pA<>_GH4O8+xI^=IAhS#v+$vmgOK=>C!~_xFg-pLM>6kUfy=zL|u~KkNJ< z$L?p*?;%(Ze6w%%M(zjE|4dH&5$)_}mG3z{KUQ6s!Y@_+kInPH;kAC&{T^5HKmqz@ z@+!aA{YNIy&r;uKTz=r6e6v>d-%9<%_4R!+-iN^8H#0N(rQbiu-u&}-|2`q@k1agM zdHkW_1&%VDD_|I;NpK*OZfAjAb z`Ttl8km0{|{F`kWKWltH$^Ech;G2y`{7&N^%H;d0$cGv7Z^oJNOSiwAFaP<=em}wX z<8AA6<}bbeZc_7S=ii6PALi)3nOXL)o&Uj%-OnQ52M&L%(%ZaWiu^(R{b!Bu2WJl< h$Zw`p^gE5e2}ml*LW4$nU|{5+pXG<~Ugg7I{||-5t(pJ; literal 0 HcmV?d00001 From e6a2e284dcf3571e75f1c1f169cca720d5d15672 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 10 Jun 2026 11:17:15 -0700 Subject: [PATCH 3/3] Added wrapper jar to JDBC json example --- .../gradle/wrapper/gradle-wrapper.jar | Bin 0 -> 48966 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 examples/jdbc-v2-json-processors/gradle/wrapper/gradle-wrapper.jar diff --git a/examples/jdbc-v2-json-processors/gradle/wrapper/gradle-wrapper.jar b/examples/jdbc-v2-json-processors/gradle/wrapper/gradle-wrapper.jar new file mode 100644 index 0000000000000000000000000000000000000000..d997cfc60f4cff0e7451d19d49a82fa986695d07 GIT binary patch literal 48966 zcma&NW0WmQwk%w>ZQHhO+qUi6W!pA(xoVef+k2O7+pkXd9rt^$@9p#T8Y9=Q^(R-x zjL3*NQ$ZRS1O)&B0s;U4fbe_$e;)(@NB~(;6+v1_IWc+}NnuerWl>cXPyoQcezKvZ z?Yzc@<~LK@Yhh-7jwvSDadFw~t7KfJ%AUfU*p0wc+3m9#p=Zo4`H`aA_wBL6 z9Q`7!;Ok~8YhZ^Vt#N97bt5aZ#mQc8r~hs3;R?H6V4(!oxSADTK|DR2PL6SQ3v6jM<>eLMh9 zAsd(APyxHNFK|G4hA_zi+YV?J+3K_*DIrdla>calRjaE)4(?YnX+AMqEM!Y|ED{^2 zI5gZ%nG-1qAVtl==8o0&F1N+aPj`Oo99RfDNP#ZHw}}UKV)zw6yy%~8Se#sKr;3?g zJGOkV2luy~HgMlEJB+L<_$@9sUXM7@bI)>-K!}JQUCUwuMdq@68q*dV+{L#Vc?r<( z?Wf1HbqxnI6=(Aw!Vv*Z1H_SoPtQTiy^bDVD8L=rRZ`IoIh@}a`!hY>VN&316I#k} z1Sg~_3ApcIFaoZ+d}>rz0Z8DL*zGq%zU1vF1z1D^YDnQrG3^QourmO6;_SrGg3?qWd9R1GMnKV>0++L*NTt>aF2*kcZ;WaudfBhTaqikS(+iNzDggUqvhh?g ziJCF8kA+V@7zi30n=b(3>X0X^lcCCKT(CI)fz-wfOA1P()V)1OciPu4b_B5ORPq&l zchP6l3u9{2on%uTwo>b-v0sIrRwPOzG;Wcq8mstd&?Pgb9rRqF#Yol1d|Q6 z7O20!+zXL(B%tC}@3QOs&T8B=I*k{!Y74nv#{M<0_g4BCf1)-f)6~`;(P-= zPqqH2%j0LDX2k5|_)zavpD{L1BW?<+s$>F&1VNb3T+gu!Dgd{W+na9(yV`M7UaCBuJZg1Y)y6{U}0=LTvxBDApz@r>dGt(m^v|jy&aLA zdsOeJcquuj3G^NkH)g)z@gTzgpr!zpE$0>$aT^{((&VA>+(nQB!M(NnPvEP}ZRz+6 zE!=UW!r7sbX3>{1{XW1?hSDNsur6cNeYxE{$bFwZzZ597{pDqjr%ag85sIns_Xz%= zqY{h#z8J6GA~vfLQ2-jWWcloE5LA62jta=C*1KxAL}jugoPqj4el4R4g3zC4nE#2-NeS{c3#!2tIS|1h8*|kpw2VSH9OcIQZx0Yh!8~P&p}fI$4Bj9Z zr5Yv?i-PfO#<}clM>mO(D0wHniZZdv8pOuJFW z+-u}BH84PQCgT~VWBM88vtCly1y$uEGJ<7vnW%!2yV>l>dxA0X0q{cN6y3u$8R-*f z-4^OlZ1HmxCv`dFW%quP<7xzAbtiFxvY0M1&2ng&A}QXAVR=prc_5m(D+_?hv#$M^ zG#MQ#fHMc!+S%HgU^Qv7Z9eu6eNqpSr3e8(;No*YfovbJ;60LjCzv9O~^>gFKO>t zGZg9`a5;$hksp*fHp{7&RE@DM&Pa@a>Kwk%*F7UGO|}^Z0ho1U$THOgX9jtCW6N$v zLOm}xcMBtw)CC(;LLX!R9jp|UsBWGfs@HaMiosA3#hFee7(4vLY}IrhD++}>pY zo+=_h+uJ;j^CP*OGQ9$0q+%}UB`4`5c766d#)*Czs<91wxw)jI^IdvyjT%<8OqI=i zNn0OUqW#POg^4ma)e2b?*Xv;dri*N0SJ7_{&0>;S!)!YV1TQuiT1C3ZFDvThe}yTCmErx#6yyQ4X@OAbHhdEV!K2%;7J>tiUZF)>Z|eRVDwtDC~=J z*M8|WEgzsyNH@-5lJE+P6HrurgY!PqtWk z^69SOHZ*}xn|j2FDVg`qRT}ob*1XiGo=x8MDEX)duljcVO}oJjuAbB$Z+f&!{z3k< zO6+{@O#2^s4qT`6k}Nw?DKV1DU~}0jVA)(kNz$c-p`*FNG#Gb&o?ko70F||R^y*hD z6HD|hJzF)G&^K=vuN$@b2fIfHVFw@hC_-0hPnB!1{=Nn~ran4VeTMM(Xx2A3h95U} z&J#Kw4>*V(LHOA<3Dy{sbW-9k5M2<%yDw~ce0+aez8 z04skG8@QEESIL;m-@Mf_hY!)KkEUowHu(>)Inz(pM`@pkxz z1_K#Qs6$E^c$7w=JLy>nSY)>aY;x2z`LW-$$rnY0!suTZSG)^0ZMeT#$0_oER zfZ1Hf>#TP|;J^rzn3V^2)Dy!goj6roAho>c=?28yjzQ>N-yU)XduKq8Lb3+ZA|#-{ z?34)Ml8%)3F1}oF;q9XFxoM}Zn{~2>kr%X_=WMen%b>n))hx6kHWNoKUBAz?($h(m(l;U*Gq7;p5J{B;kfO^C%C9HhtW!=O3-h>$U zI2=uaEymeK^h#QuB8a?1Qr0Gn;ZZ@;otg2l>gf= z$_mO!iis+#(8-GZw`ZiCnt}>qKmghHCb)`6U!8qS*DhBANfGj|U2C->7>*Bqe5h<% zF+9uy>$;#cZB>?Wdz3mqi2Y>+6-#!Dd56@$WF{_^P2?6kNNfaw!r74>MZUNkFAt*H zvS@2hNmT%xnXp}_1gixv9!5#YI3ftgFXG20Vt1IQ(~+HmryrZI+r0(y2Scl+y=G^* zxt$Vvn&S=Vul-rgOlYNio7%ST_3!t`_`N@SCv$ppCqok(Q+i_?OL}2@TU$dr6B$c8 zQ$Z(lS6fp%7f}ymQwJAIdpkN~8$)O3|K7Z;{FD?hBSP-#pJgq0C_SFT;^sBc#da0M z;^UuXXq{!hEwQpp(o9+)jPM6ru1P$u0evVO(NJ;%0FgmMNlJ+BJ zf^`a|U*ab?uN*Ue>tHJ$Pl~chCwRnxi3%X06NxwlIAKa*KReLL^y1B^nuy|^SPj3} z5X|?1divh3@zci;648jb2qEOm!_8Tjh3gi;H%2`d`~Q(IL{Wcl1C18+&P>tU&0!nO z&+7mpvr2SsTj=@sX zxG=;T^f7Rg=c=V*u8X(fo)4;RYax^+=quviOJ{>r6{wgf)g){I&qe`=HL}6J>i6Ne zSZ*h9f&JG>Y`@Bg5Pb&>4&UqFp9I<8o`n4W_V=4AugM`RqUeS-!`OyNLyKMqa_Ct| zON-hyk#-}{lZZx>B1F@dF^8S>x|C*QAjKqn&Ej9H#z@Q#KA*ckBX@^;gIP&?aK15l z*EY@kG57oUcm(d{NyXg6$Kj#xR5XdZ1EBCT+Zy!gyXwN&b_zI&$$>7R#{ zh8U@H8NY-cA*CBfH$OCs^priPwtwrzFjDO}DBn#mgbI~hn}cp2U{yv@S)iy|jR9+E zgd(hF|1cyC#te0P;iFGqpNBqc(k<{p^1>wHE_c8Tr4|&NV4mzpzFe;Cr)C~qpVNjl z^u(^s5=kj{QBae)Y*#^A39jT4`!NuIUQzD#DOyfa!R=PrX6oS@x@kJV)Cn$!xTK9A&VI#F-Slt8I4|=$bcjaC5h=9E{51g8X5q1Qfg~~G>qAgy*7h4-WuqE zlIEx?Hu*%99?$6TheLAD4NIMO=Q@*;gaXDl6yLLXfFX0*1-9KQm42c%WX*AXFo$it z?FwnWn2tBHY&Qj6=PV?ergU$VKzu+`(5pCRqX}IoSFo?P!`sff%u1?N+(KsoL+K={ zi*JGl%_jiuB;&YW+n%1o^%5@!HB9}OlIdQZ*XzQ%vu!8p2gnKW+!X>@oC{gp3lNx^ z82|5Jdg9-B<1j|y(@3J;$D-lqdnf0Q6T~q7;#O}EMPV3k(bi$DpZwj9(UhU%_l&nN zR}8tN_NhDMhs)gtG*76~+W2yQ{!kDTE@X4gft2?W;S$BLp9X z;sh2jpm!mkfPX>Vuqxyt76<@f4fyY%&iuDfS1@#PHgzHqG;=X^`X}t2|Alr^lx^ja z1rhvG(PH(a0THitc?4hk=P*#IS;-`fjOKqJ4kgo@dAD@ob*))H)=)6s3cthp&4Q55 z4dQRdG0EveK*(ZUCFcCjILgS#$@%y=8leYxN-%zQaky@H?kjhyBrLYA!cv>kV5;i1 zZ^w&U7s&K8fNr4Pfy9GyTK2Tiay4Y_PsPWoWW5YA8nfUkoyjU)i@nKj@4rY13sxO6 z_NzYdG=Vr<@08Xi#8rnX&^d{Bl`oHXO6Y3!v2U~ZV>I*30X3X&4@zqqVO~RyF)6?a zD(<+33_9TqeHL)#Y?($m4_zZvaJXWXppZ4?wo?$wF)%M6rEVk2gM=l9k+=*Q+((fI zIUBH6)}M?ahSxD4lgmJ30ygk#4d!O@?%WNEONommx`ZK81ZV)mJpKB`PgQ}F>NGdV zkV|>^}oWQd6@Ay7$&)6!% zOu_p~TZ3A#G_UqiJ85&*$!(+!V*+*{&-JXb53gtc9n3>8)T$jUVXe+M6n$m633Mi? zlh5{_+6iZ<%gMWMrtHyDl(u-hMl^DViUDc50UD;0g_l$F`Hb(F=o+?94B0fjb;|?Q5c~TWX>t8i1RP@>Ccgm z?2=z0coeb?uvn44moKFb^+(#pAdHE7{EW(DxJE=@Z0^Am`dpm98e`*S+-~*zmhdQ7 zCNig0!yUu5U#>KKocrg-xMjQoNzQ`th0f{!0`ammp_KMFh?_zF4#YhF35bPE&Fq~_ z#VnniU6fso{!3Z^1C57q?0i!ok(a zL;-f$YlDk%qi%n637_$=Gw=bBY}8#meS~+#X}Oz~ZKd%q(UE>f%!qca?(u}) z!tLTuQadlAN;a#^A?!@V=T?oeJ1f7yRy)H1zn_+wARewYIYr`zD=^v+D|ObvH4rOB zT@duqF>$Dk6&i|pZh?%Wq-7_kyP4l)-nqBz#G0lqo3J2D%zmbU)>3)5e?sTZy8|~B zPC7!`eD+deR?L6$6 z-e{!ihef=f<4HPZ9rSt&yb=5Q)BFAXWPR^~a&Zru?8146wvlm;<)ugbd|!}O6aE0t z6`#KqcH#S#*yz-K90+!Fhv+ zKH+?!_0yl|gWXSaASLcB9a8g7i%qz*vbO)YW`Q@Nxpp*6TZ*OO8Z|5-UWihd@CUXF zY!aTAZ$c^?4hiaq34=s2il}#Pxu=#c2^=(PbHNAyUqy__kR+n?twKrQe^8l6rk=orf}Mk80viC1NZ^1q zeF~g*iGp0=jKncK%s@#jZcn6=EiR<8S#)yiEOuwbG;SV$4lB^R?7sxOf8)oq$sT)) zA&nBCFJxsnci+)owdCHV#cjP2|1j22xIRsxHrLLBk3GI|OppUv3%r>#;J|26!W>xC z9gq@NQWJ`|gH}F{-QG#R6xlT<;=43amaDT>VaG*;GfPZJ&W*rO8WAQQc^JGw-fz-| zzAe&RAnC(gAP#FoJtt~ynR3Z<)m_<9Oo)XW}CWd50^eI4!1p4}s(zLhBIDi5r zr{UH>YIz2!+&Cy(RI(;ja_>SUC2Q`ohWPlI+sK-6IU}*nIsT)vLnuVPFM%~gdel}S zUlY%>H$?-rQRGTdUM^p^FEkqnwC{^BGl|gM)h9zkXplL90;yOcgt(8&LJwOj!5Qgy zu$@^*k%9JoAzwj@iSB^SNu#YVl@&*g$uYxxsJBvIQ>bfuS97JccQcS7&a z)`1m2^@5c9pD`P$VqH*O*fxkvFRtH-@Pd0@3y2!jW>i=jabBCJ+bW@wwUkWjwx_WR zHH5*XR4hbQ1`D@4@unmyEX)!?^~_}~JQNvP4jO&F)CH9srkFhf8h*=P z;X1&vs_&v03#BGc`|#@!ZONxVj9Ssb#_d63jxA6dX_RBt(s;ig3#s(YU3P3klF;mc z%%@^IJUAlGE=cnsTH+(qb1SxN@HzfAjYcUCb(VU)JV^3ZC;#k!t?XjaC!|68eLE zU_hlvOSNj7Qlr{x)y$S$l^2DPCMA=pzapcSkjfk*r!iWU%T{?<3#Hw6s1ux1^Ao6o zR@5DIfo-|c9AaFw848Y!BVG-+vURe;I29F#hLu$9o}oSa9&2sgG#;lj@@)9|2Z3 zon?%NV&AYSVnd~eW~v0yoF$X^1FR@i2kin0mFLG8-aA>hYK;B%TJ~7%P4?_{Bu<0t zvmI)Uk-MRncVb)A890>OqnYf=wu-J5A~^%4jpK~*xp)=h0BZB4*5uWrP>iRV+|kMX zv+BEskY~(P-K)-!JSHR`$brY)HFI|L@YyrxheT3cgHu}KtF%s%k3B`X)E_lA=E>M4 z2VV3M{c0*)`qZAsJ==)F#D~2Ndzm@hKhSBL_Sf3{ctckh-rB`gkfC?Dp6FdM?p;vv z#UlQMp3H5*)8o#Ys@-aj7O#brUfgQ7BjG`7 ztoE7v-tH2%KVC$xKYf%uvZD!_uf3x>h?8r!zYHkcc7$Gdn(6cDmYL&p3pCfaSfY4$ zG|yuujr6!Wl0}V%* zQ;nY##kEdvo8YY=SVDb)M>^Ub9e#4c$O&urD$uaRtxm-UH=6_s0m^^5y^_+F^Q?;8 z+Fd?+De}er^2EmFNn&e8SyS*`*`e;KFIG&+x5iWCsrEyH*0SFBCMx?`m5~hl1BrT> zr8W3*3}Fwsx@%UOuxNoCSoL%AM{Uj|v@>l{pYYI&D$j`&**;?X`cuOOk~?;U{~xvDUjaiH^d`A+gQL#Z?*lm)x_n6R-S% zf6*=Q1m>mq5|Niefl8s=5F={ncn5S;6~&Ns2)yGZ@wt&u4c+)Sk?hdfI^b77@K-=y zM_k=j5hp&u`2nkJK+2Lw`uLypr4dO?Bm3BTZdtWnQa5unCoTKIiG81t4bG`epBU5| zG{toT`)LE}&j{P+AFj`YZrjF-^>k+`zCM`QcQz^Ba4BEte@S}j=Q_Opx14jq|DB}& zNB44BOJ`?GJM({v`gh9pzbg8-%Un=E@uLfJwGkagLEM^!`ct3s5@-xqq*xd+2C@eu z*1ge`retZK)=bPO<`>@62cLN?^S%v#EsiPQF`cg&I7{}l?)}O$!^wNJp4Zd;1yBbQ zv@_7x7d6aXJvGHkNNcOg?A};m_Nq7H=(+zqf9)e3&yP^EU63Ew!NW4CYj_!=OTVb* z-ijSrv0M)u=MF=@+`3ldT-hzOn$Ng><)WL0vqQ&jH>W7EmLLQY+c?%i9~f_x&{OYX z{?kyyNZ&gT*m$(%-OeDAJeC^c)X!k${D*c;c}9)0_7iWMbfu)!j3+{*!Dj|?C`sGz z2xWha)#`9@p*{-X2MN2a;%FM-WqB2h)GTqQH$ZsGD#Wi`;+$i?fk;23fLpYI^3TT3 z5+Zn3cu-_2Ck*@%3^L3}JpVN`5ZJ;gmKn>gm(Z)b%!v|RYf(qrmGL#0$WHQFw4mJqQ85w=$tn^7(z|eJ$3R0} z2k9^EU<^-$ygq!ZR+7wT0KViK8qkAO7xs*e@1dq{=M3haulHwA0~BYNytr7k2K*(W z755P9a^;Hdl2X;K{c}yWr|QH?PEuh6x)9n{^3m2QUfC_Q*BW&<9#^ZVwOolx@6y9- z-YF=S;mEypj68yxNxfJ56x%ES`z-5$M${V1HX(@#R>%$X`67*Ab8vC6UzvoDOY*P= zFbPXany0%>rqH1gi7d>e`=PWZTG>^=#PQf&iJjJ0&2dO(4b8) zCl%8xJg1mg4__!?t|y_roExn~%u@Eu|p9YFb`8_qP@v#KW#kFs4eVetJ+Q+s|Y0?#D z@?dt_BA7C4tGpjOB~*LFu0!5oU(_xj7xA$meN)Z;q4Z_Rb7jY1rJBzJPr0V=(y99F zh=V-NbK+64rd#ltw~7X-%kP$R896DxRuj)p7Zj@8&>IlP&}ME3s9eV2R>SpUnSxeg zmpm?HQJ^u1T;pvwvlc4F_)>3P~jlTch4+u6;o{@PtpnJcn~p0v_6Po%*KkTXV#2AGc) zv)jvvC?l#s$yvyy=>=7D3pkmV24xhd7<5}f_u5!8gmOU|4555dv`I=rLWW!W!Uxg| zFGXpH3~)9!C2|Y6oB~$gz(;$CTnw&R&psa+E!KNgrE1+WkLM6SOf$>sGW+Y{>u?Fw zTc!xG{pa3c#y@d$d0e7a9~e_xjGcaw5f6Fk>lg$Jm}cFd%BO_YT(9s+_Q;ft%1*k$ z_cXkf&QHkaQr9U?*Gr$r6|bCV>2S)Cedfk3rO?JbyabY zgqxm#BM7Sg6s-`5%(p@SxBJzR6w`O6`+Kuo36wwBzwf6K{0HENVz^^w|E$r zdZM%T0oy8OK|>>2vSzw5rqoqEroCZ%(^OmOSFN84B2-8Z?R1)Pn9|5Xkui(fQRl^zA35EH^(JbuQd@Uh z2FJ6C(5FDD(++_NLOG)1H<+X~pt68d@JiB8iUQSZ+?qc;Jr+aJ8bKF3z`K&zSl&C7 zEgl&!h?sc=}K7 ziEC(3IrY?h7|d= zVjh{@BGW^AaNcdRceoiKmQI+F$ITdcM$YigXtH)6<-7d@5DyyWw}s!`72j`A{QC~e ze-u0a6A;QSPT$vqf3f(kO1j^%GYap*vfWQ@X=n{lR9%HX^R~t+HoeaT5%L7XSTNn` zCzo})tF@DMZ$|t6$KTx+WQqu~PXPa9FL&shBGx3C>FlGz}7gjfv}(NKvjR#r5PL$a1>%asaylWA8^g!KJ=$}_UccHmi zAZd5c{I&Ywpi3a1#27C6TC~zm3y8D>_1an8XHGNgL?uT$p+a<5AdWLR6w9jdhUt9U zz?)93=1p$x;Qiq!CYbX&S}+IITWLkfu%T6X5(pk9-fs8lh9z8h?9+>GlFeFcs*Z>u zJSaL!2?L8LbOu_Ye!=4~ZKL?643lcsNn8>qUT|q&Rv+(z>Z9=tyG&5}zZK&Q?S!nG zR;Ui^<406=jLYA>zl!a-OXH#J-pP4A`=)r%9HV5m1qGZ1m*t^wi>3$JRcH)3Q(LQz z(3}~y3=QsUu!PN$$N~#yBP@=aJ+Bkp_hx8^x1Ou6+(Kk9l1CXr4p~IQvq@AUePuAj zcq5>YDr(JTmrAuLwn6sgohTR-vc^y^#I{grF7 zg}8?&5!^$|{X`C;YrZ7?rKH#`=n0zck(q37+5%U;Hmds2w+dLmm9|@`HqQ<5CUEz{I1eNIL?X~rd{f71y z>_<94#1G+j`d5|fKK@>QDK6|HRR|9UZvO6HdB1afJvuwUf8bw>_Fha)Ii8I}Gqw}p zdS~e^K4j{d%y+A#OBa1C4i0)sM=}tjd8fZ9#uY}{#G7rJp{t6?*5*A^KKhim06i{}OJ%eA@M~zIfA`h_gJ_o%w;FaFQMnVkBT|_ z(`m9r+11~EPh9f7>S=$F7|ibj=4Pt>WVzk6NfGRvI_aG66RHig-(S%WKRLP%_h0He``xT))N^RI@6!ADl=*vsqVb|7 zr~Lwl6qn|u!%is<{YA`Mde2Z${@EAHC^t>4`X;F9za=RC{{$4OcGmw%9+{$i@!cCn z;7w~r8HY->M@3OzYh+L7Z2Lc8AcP*FZbl6VVN*_sp}K zQP|=g@aFthq}*?|+Gm4@wbs_?Fx-HD2%)_UDJ);X88~7ch~d0cJ!<7;mv>iv!RS$a z;(-cYTW=K=|F0gIg3EW0%u2CSr(Kx}yLoki|KSIt$#P(O!=UjBGRzb3L3-?NGr7!! z^VC7_Q(GhT;C*(bLivfhlRDVdz7=h%ABuLA2g$qy)A}U@Kj_L-Jd|--fy#-*ESRo| zgu?*?jGEgs9y>1`t}|^Ucd1I=1N=mOo{8Ph zwZS(F%G?nfI{#%sGayNItK9J5P)Qk+^4$ZoXZJ0G1}hwcckJ0g-QJ<)3%`bF8}(ahYIjKFYMtg3X;e7J18ZvDkV@N=nxvDl zo?}lXoT3pZY;4$QKI`~GFuQKv;G6b<8;o89Hd2yu+|%sU(9C=h8ibwZ zARqZ#lk@kp4*#URe-YmpRc&=-b&QP>5b{9{(tH*)(@ZPKfOslBgwCPx6d*{XMX|Q{y0F!5a^ScCE;h8bQmTJR3*}A>aGcDF0?tU)Tnml z#DgruwAva-fiU3s*POY_ZHiJyW%v+733X`&ocwHz$uqJCOhrM;#u*V2eK$D5HiN(` zII{BEg(PV6#_Nv3rZBUyd+TI!>L72KW_Oml6L=pNv#aOl( zgpYxAH^@2aJQu3urlrCeanwSpHHD_Cxb+=cm49{ZU5Z@;{^{okEJ6&fpDD31w~$`% zcz@_REsC~Vq>3YF7yJ41ZEPBW&%|OwlnfG|QNpiX;fGR0f^3?PEf|-33P&LFGe`8^ zaX3M+*h+?6;s|=$j*d|S-r6PSHnmLqm9oshPNpGzlxV21cFrxcQLidd2%h>n%Mc4{ z|JWBvtbb;(-nhWpPO95hR>(e(H$n%*pCh0k4xE#I%xu=#B)zXSaH+azwCI;0@bY<*-10-Qyaq%5NxSlq_@YJUUwy z*d;qPjW^cuKxdXiOWwP}5FN6SZW~NqB%4?|WifPNZr&XNVkzF0n#Y)pbaEodqNO4F z2Bq#^Gr^Ji3!T9`_!D;a1lW$?!LQ-iYV_A{FQ~^C-Jp`_5uOC)6+mzBr4Nl3fHly% zcXeU3x-?#J`=p$6c~$T~V^!C0Bk_3#WYrtoFCx9_5quCQ*4*?XG0n_9%l_!n`M85^ z7}~Clj~ocls6)V&sWGs?B<`{Ob>vnbXZwdda%ipwbzOJ(V`W>KBF5zdCTE8;mc&xU z^clCzd0(T#8*(})tSYSNP1N{FnNVAU^M1S_pq4VEQ*#5nv`CoYSALMEB zf6egyuRMzK2?r^M0hCD*sU;On6c0^Vh|#tRG*n1p5R)QyVw%Va37nMSV%9&uq^hp| zCHeu}y{m=NsA=naDy;q`fd9t)I$Qd-A1Il$#0KyDc>X)hKJViqNB{HnQyf5D(ZJ*J z{-oGB-%Q|QZ%Pqu34>fCy)Asi}IY7luNR9ebgH4DAjCVvSWfa%PE16 zkC7EIuEK}?IR!jgP%eX%dcxk4%N!zIjW4wYMfIq@s%GetDs^g!^p}DH46EP`Nh_wD z4Rwc4ezh1U$Mc)Fe6ii6eD^*iB2MFp-B-HhGTR0tC2?bq$#^J!v1r+Z0y+& znVub*k=*^0yP(c#mEvX}@Abx%&}!W(1olcWEHAVgskbBrzx(f2v&}4~WkVN?af#yi z4IE-(_^)?4e3(d{F@0<~NV5|e0eaB!?(g%l&Hq$UqzC_Enuest?CL+IrSD`tv8|{C z=79vnL=P6ne+}6X1&cd$kam=jCcv`~^y#R{doTh?6D?H)^M7-P+=D@?H;bt$*V+)K z?+?Ex3Z@8JE3c4eHDYItB^tSot;@2p_fuZ8mW^i^a(L;Xn6K+1GuG0n$v(38;+<78 zC?eMzbQCW2%&;U>j}b>YEH5>RkP44$QlG6k(KwXtq{e#13wnx5Jh=uH?lQIl8%Qxr zq%pDC)mYYKa?N>%aF%YwA}CzV@IOV9&a81d9eiU-6F&lGvz68~%{&4LuwV_5{#km3(tf`fejjs%`{Y`|0p!6|-U z8XQA9Sl=*kM|(2KA!LWOCY3Qq4sZ7r&}__rR*Sj(9W8R1_RxI&4TI+_7RSJF&-363 zJvczH?1(`Jb+RDJL9$Whnj8qJRI+Mz9=Qjvubb=Lz8nWVXG{Te;$%s9-D#$)-!{~w zIM(vkr#OM>2F7W$$Lq%fEYl%e|Tsc>9rB9c8 zQoi4nXomx3&sBI9AwaHkoOp%SMDf2@T#73Bi?|!r!Q?wc(^b_u4ranezYx~=aRV-a zD|_WPK^iJh&=)~h{t<>_$VMXsee;{r-|`#H|1?DZgWvuc*!&C2*(yv(4G5s{8ZRzt zZMC~5gjiU@6fPGMN%X~pL};Q`|IfPfs0m9;RV}xSxjb)*gmvGO1`CQb~W1M1{KwXBLyPz0JQG=JkVX zlPq&zNZS59gf-?*5Z0IFitTX4T$1Oo#_~V%4q2vI?Y@UkSHh}H9xZ1va}^oBrCY{+ z3wwj*FHCsS2}GdSG7W(|k+MWu9h1Qs6cft~RH)n*!;)5HmPX1DqrJ3-Cs%i4q^{$N zC&skM7#8f{&S!9Eq-WqyY$u?uTgrSDt#NU%{3bQZtUSkUof4`Z1P8aLOKJ+^dKh%n zfEfQ zO|P*J>;{=`9@D)qpnt`#NH>}sir*&oFC+W!HR)ecHcPwjF-|)}8+tR#@A+~CLl+Ab zCqp+=Cuc(&VGC1ZYg4CxIXYL>33p^wjIWJSh6R=oq)jD52q3~KVGt=w_z(arS!gx^ zSd|?!rzDu1$>0o0Y0+!iZU=ew^Hr+cq(I(C>9}^sBc++0+S#I;js@_NLD9>MH(tN3 zE5F+J_bYdPfYm5%7-e=lm?!-xlvX~nDkBqu!Zf0ra65JD&@tYDW+c@P3W-YyWe4^6 zhW?FUJ;c{^?b`N)03>!@#JI)r2&!6An27q?*^wyUx3T4uyeIl4*(4CV5OTK#RSnYt zq<+RKCdrYIJtdmNC-NtfH)K&pytbM^Mi6JWjkzJo0TdX>HOjJaIQmQ?Q;l2)8oN@d zVyT=%y@TihQaJX7#B2wY#_ufuaF55-sWO{OwUx$2zRyW$YM(CFBs4Y;YmBk(4u&u- zEf@rIR~4#}IMeq$?T%z3s3RAR7m%M?8No;a=1HXKP?ia#uwy!`4v0GFSjZiMii@ib z#xRmA-v~CSVl8z9cEWVEk;9_BKPS6Y2|bk#PAb|}gPxHs-dt*k`5tU#FZL)FLodY8 zmb!m`DagEJ#q1VKwO~%zmw7;LESf5u!KJNm829pbY_w$P2}16`Bb?0uoL3~V71;_U z`B~wKOB7Bp!Vn!M@o?RHydmah!dHPaT`&idV83kQPxA>E=~YgJC<)rdM1#B$JIgnq z0V{p|Cm3eeMaO58Wrv^9-kAOJ+*HR!;;A9z&>78VsYmF9$U^*ZE=K%d7=MZ~G?~Hz zSHlKWK!Us^%?uE6`E|_XI+nC354jkbUPvedHbh(DkKGkquYf}=-EEB1g>RC{O9ORL371y8V*CR5EW z@lmFq%MWEBdeHR7%(Rpf!Yg52vX%D7#@*^M`fy7Srb z^Ta9wcwf$89uL61@qeg2vc&TAGKSLV>YKI3#5lfs#q5Zm`~Ogef!!CoWWyiA=J;js z%X_n!njeF2MZgaVoMh@S@8%lR)AsYyzmqkj+C8ghxI4G6O7ovK$udULO!2$(|__`2~6JjuoERet}kenJ%I0pU_O@tU*Fsd4gm&hV?p%Y{!;r}{S^Fv z_4EJbVjFv7>+dE9{rBS@8&_vbx9>4!8&g4JV^e2mSwlNR^Z&ujriy)b3jzqfYb35o z!;J+c>%LY+?P!IticwSrP;x2|k>j3Sxg2X%E2%57

`Lem|V$A>eR0uN8Y&sdjtu z%-lD<@61@6?qUPjUg|mF7!P7`hx+st`i!^L7HVHtzwnM z)LuOANIzT#9tU4)C^WIXhZWqrO;jr_O5aErkklzt)R-JmAh8xHMJ>x>OvTiuRi}FY z-o@0kFwwl7p|ro=*2q*cFRX5GCq-v!LPD)Sq+Uz~UkOwx-?X&!Q^4H)$|;=n9{idC z0mJl`tCTs3+e_EFVzQ}s`f_4fijsucWy5y zarHoT>Q06Z4yI1RPNpW`@4hSzZT|J`MU3i(GqNhm*9O@MndJ{31uA^i zXo&^c`EZ}5W)(|YMl##@MuSK#wyZ3dwJEz*n@C(Ry$|d`^D=thayXFqxt*WW&sWdI zdm1wv#VCKa<7d2Qc#qzvUvivhK5wq*djL7Wqjvf}-c~}d#G)eG`(u<`NGei`BFe4Q ztTSs?Gc8Ff%_5T4ce&J0v*FT`y_9r!Po=sPtHs5~BlV6VEUNzxU+)+sX}ffdPTRI^ z+qP}ns9yQgjY^t0ddMx1Yd`|OB{sHnUC-B;qum1|`tR#P_@llx>d z=qpNN&?nZib(t90A9F*U%1GbB+O;dq!cNgmmdCrK=(zS1zg*9(7VMfv)QMkt_F=wz zHX2p4X-R*=tJI4A)3SrL`H^peBNHh&XC#sVR3D zt17qeF>BaCZNlQO7n@@BuWs&l(FtRjaVn~wW^x-GsjpFH!ETyl7Od{Wf;4=bzL5nj zW9c^ZodMnN{3Jkz2j2;qhCm1ede*6891vR9?(Dy)N|iENw}HKLIOrjB0x)pEs-aS{ zZR$tEyZxbP(;(l43^KjRtSuirNmw~Bg&6p;)vqM*>S#L>0+Pw5CU%4@&)8OX2ykYQ z^f^hk-5%!QzuzYniL*1Gs#S5Kp_*ld1EAmkInP+^w?#(?rbC2Bm&0c5Ko@6`_ zi!Nvd391nu^@AmpZ$_0fPR2~kQGJS7lSGwA7U>s@+!d_`(P5y;MT#U~_ONSo9d+bf zVj6MgWN=|%#Qn;vl*TNLE$Mw|*89{yJ=WN>j{?T*vqa$U$2_dg46R)8wl&CNS&iK{ z>HDBC9e3b3roJd}gK!T>takKP);KLj_9T;%knG_fN^S$4hb`E|)qy__^=mm&Z{~CF zhc*PxdrJ@xRkQ-8lbh3Ys@2ZaR)Q3z**-VSgeMHE>c5AH1bpSUor&dgTiMd5Wn|(# z8Rwb{#uWZG(Jo0co98|mg5zF}M*d>gAg|Zdex@}Ps&`51({MmNyHF;GD4EBT`oP|X zd=Tq9JYz*IP%@2oujruVrK#jAT97|%ww60Ov2He^5zA4)VihJ$-bxoaqE7zU$rmK) z#O!xp&k$!TOEiC8+p6`Q)uNg4u8*chnx*aw=#oP~05DS&8gnL>^zpBkqqiSQA{Ita z%-)qosk1^`p&aB@rZ#)&3_|u{QqZO z{f{A3)XMprL}2{=pM$*`z*fY;{=4e=u7&=s+zI)ANd+V!L%#^2hpy@#N-WbB%U2Zl zgD_E0AVVWdMiFi_u2qqxeAsRzD%>l|g-|#$ayD3wHoT{EUS2Qe zEq=ryLi%iMZ`b}tSYzHInTJ{mY{OXy0)T&Rly3ippqpTk%A{T+e?K}j zURM^%!ZIWxW$32?Z&q9)Rao;#KQuLv+^ft>o|6c@QD=_}ql%5Th=cR{P)_51Qxjh# zRJW<|qmpRn3(K1lMwU-ayxjsgKS`Q7J5m0kw|LQb=CbyahnoQTWY z?g8-#_J+=*r`Jc|A0(MOvTc0kT-tBLIIFCd6Y5iCr>cqubJu0`Ox+FkDWs^L{;0mc zxk-nf?rxh(N<1B;<;9PSrR4D<*5!DvA()O7{vl9sps3x_-Y_w>qC3OI!_Wyza8K|E zAvJvWYyu)(z*TK7e+Q#dFWd_7%;fn4Ex*lEY2$X%SP9K9d6yWC2M!3>3>tu}g4R*V zRMC!~oYyF#Izu$lGjfQ?q}KD$rpDMRjF?f>6kuBlE`z4Yxy(Y(Y+Dr#PKA}UsSWD? zm|ER_O==Y22{m%cO1jhu`8bQ05@MlII86NP>-_`<|Q4g1f7Jh*4%=yY_ zafIlUJ2zA?dT8&WTGLE&gvPl|<0zKa=DLzzPOU7i#nate!Z3u|9R6E(6FZ|(EZ%+b zsB!MEkGz1K*oXGdp^tGOWyF0SI{tq>^nbgX|L>uTert_v9gIv#Ma|5OTy0(c_qQUz z!2+;T+eysD^IV+aC=aX$FPzbq+lZ7Gsa%r9l;b5{L-%qurFp89kpztdmZa8Uo!Btl zu7_NZMXQ=6T6+OFOCou6Xc_6tf!t+bSBNk)mLTlQ5ftr247OV6Mc0v+;x&BNW0wvJ zjRR9TWG^(<$&{@;eSs-b796_N#nMB4$rfzYM1jb>Gu$tEpL8-n>zGXVye2xB-qpV z&IZjhW#ka?h8F{QJqaK&xT~T;$AcKQD$V>$$-$x~1&qfWks(mJ8#7v7m4zpWw(NS( z5j0d&Bs4g)>{7yzl-7Fw`07Sj6{vw5nwVyVt8`;Rg5bzISP26=y}0htlPKRa8CaG# z=gw7__ltw`BWvICf>5(LFDFzC7u-Ij7*OKwd7685%wb6a=QD1CjpQs$^2~cx`@xS` zNMz6?Q4OgIR8LYa&m`q*QJ%!CbD#=ha?38!M&7yLA1Wn}M{$nV3-G0@@bD#WjCYI) zKFZ`bf$tFF#}GYZ7MK2U4AKI-GY*y(&DCt~4F1!3!{>cK+7XAfKw<)Jv$b1vHkpC;gl=VNy?f-RI(r=&j z@Dy@&vHYi$GBI*-`1j-=qpI@{qwt%et&>`VuG+PYzF>DUM1!h|8sz~*0>sA7|IH_y zskL`MJ4Yw|Ru~}gzgCOOEDSyuM+ivsjt@13h-SLD|INP2zRO|RKEDz$_zlt)ZWYQg zKHk`_;gygz9b$7*)WKC(<}zQUY8M94a#Tu_OEyX$Lej=Cs`b}zjTYvv-Jt6E^_bV) zCt>gvm2{y2tK8Uy*;ruhTa_?lSIlV;r8b zX?jME!z32pO8`g9ga%`RQ*v=F0O`bnPZebx@b#ZfQWvqZPAb@zl>ORo<_o7Dp&F?6 zP(tBH@~c-Zfx?Ulkb{F`C1S8y3F;;)^MwWBiBPQ1D=;yC{M-i~ILSfh3K!Ai{5c?J zdLm0OmDsWuV>%}MT*Qf<$UT+M=7pMVdJGRi-rdW>7iM&2UO%v@>_!inA`JD)lrKC& z75Y)Lg~PVq0Ge}-g$8cy0w@sHjUuwMm1|~u6X!*fGG>%bAbv5cEU3nR6&6o03J2ff z)*M)kj|gyvZ6Md8Y!m#IuWuP0<9daW2gPDp*=aQA2qm)VLJ($UUQ>-4&3LX|)=-g5 zDTzngTm?JwMM46$Z22o7jlr3Vp3K15k^@=c7JJx9WQg*XbLRkdC zYapmoZr8J8X5n5}a2xjY35bC^@Ez{}9JA&aex@>JiMr#&GtJGn$)Tt=HVKx@B+w50tPaNkh{N0!^9>r<#h(fr3kP@a(N1!O)$rdf&Dd!hhJNtXD zIbx!f3YSHV50oNza38Kzd9Vze|NZlyBd{fKzZOSB7NqO*qDh)*>XW~VnmJ^ zji(MF3D>tHCk-^y37b-c7t1Zrt)VBlefNnY+NH0u=9IPbDZ1z8XbK{5_W?~aGs@o& zTbi2gdn~PB;M%^{Q*d9xWhw;xy?E}nCbBs0rn@{51pJ@6e=LQg2dvlq_FM0;Iel9= zz?V~4Y+a&wJIgvt5@%1FDtB9(A<-f!NpP^nl51v_hp$v8$w{ z=Rh2*Y?stNGlx7wbOLqrFbxg3lqpaaN{@9c)nNxe#D=Xouh@g7Wd}stZ!B8jrc4HPmOW%Xt^a!LcN8M4^efD8wWziBkha6&KggDq^9beRoiLH_z9 zGUiqkIvsoqX!3F)6qr+_HfB$D%@)T=XV3YUews|Tg-Hwn^wh3)q=N>FC*4nHJ+L$K zpR;I6Gt%?U%!6mxrP$mlEEiT&BVf$x(VJRuEIXdqtS+qfX^-@UKefF=?Q z(jc2Y2oyEyr3_bP|F%)C?~RzdfbNXgw%b_zaAs2QbA_QL+IyP^@l+{#{17?2dn80k zljl~W{3$~wO4E?SSij&`vnbpKCUzN%8GY^!-wNR8=XKiz>yng^Xj99@bTW|TDw5XGfDje2@E z*~-mJF8z}cI1eTpHlg*7?K(U5q3H%{y84gCiDbksT+HB=ca!YVTu zgPDuJzB@76rs{is=F^_95WD#mg}F*~wRr~vgN4^*Gy=hUUD_~f0QPh!&J7XP9zv&H zY}Zm4O#rej< zQmBNK_0>1jXd)Y3cJi(*1U|!mL(;nU#j_WV33)oK-!s$XS(mQqWqQ7&ZZ54iT5+r| zi|MH>VJs`1ZQr<{eTMqC#Y~41>Ga4BuQynUV!QuZeaFa6aP(B)SxC~V-r0K5 z5BJ<3nuAkX12%0k5qI=#D*PNg{NNjn>VUnvH!{DfD}FX=e%E5lw-IZgDqD$1an(zv z95TXS9wGg?Bl{w91nOC8HvvD1&ENr~L>4u{^bNaBD>ZHXIw1Ko!;wjz1%zZMbWE8# z7f5xlDTQWK%rH+)0KY&O>*EHs@Ha5t9ltEE{qv`K0tO?W=jgzciZhHZ4As;i<7{@M(!#&K$4UGQ?~d6rbu|rCYd`D!Bgha2*v# z?6){N62Wq7br9`S=y(rk$xKExQsyv0H~Z<~f!Z7~Wt6SlJBO4_KeNahC?2rxh%Z14 z{6vx|=@Pd?8vwjCEbf?V*zgc>36eg4u4w8WMluPe+qB=i60{qnN+XKmud{LfKvd^Rf{8@jDa#RaXtvGeC92KvnMDV3m2 z4Xt7QB96VazV=Z?RrMXb$#mb85@y7X+OE;c6PL94T|ssUhD|n8IM`GhqU%%}=6E(! z@O+LF*%Uy084M_#De*pBSU<)G3|%go1vt<|<(ZKk{3&*44f?ftxS-a(+@u_92o7ot zYq%I+Ztyt1x5RPt_1it>&+05XbK1B{-T~aA+FN6BiF@>|QCJ`#y*u z@e*p+J|+Jzl4qtDnLJPde6Gl8Qfu5eP#Lr_}cyBzGaR912ca0h5s# zbgocm38uvIstvyAPMEgVj^>{XqR&db7$(XJRTRiR@!lH>>CTe{+zRJEgcn{?M627> zsw6}Y)J+s3)u#g*Mo19)oWp785&T@;fee1**^o5#bgS4epuPWP>~Y2v-~{)-me7SK zd!AQUXsd{A=;C;8>vRTE5Dol&>XJ&AYMijyXV3|_46Fr#lz`uF9dT^PhX2e>lDN?r z>wx*9-Pr~siloVs7@`dn*kGmY0xP)2odnz6S437Hi&}MSb1iiwEiwfy=f;yg# zDZojIe7{n|lnmh@$rU>6-%oUGrG#^0y%z_Niq4LG38Yq&Dq<~B-3qLMHLbL;&A)i3w zq0}L%{J2P1a z2OC$%f4j5C`~!#oBU=IP{19v?%zqxLR77sUDKZWk1TEdClEz1yHB10F7>l{;9l0L|=ADc&?i zK#F90YE|)m(u4LGC%M^0?53NrH3M`xl2{P!5+fC(H)Yt|t=X~m+os4b6}Wj|nDvL8 z8n=Bhi`Mq$&2sm(8n4F2)~_ylMf-R2rn!V)Bfzhv7v2SF{79o}>ITpgUpe=zcRpds zp^3fse>q!&ohi{7gYJM|qD$1?s^vyP1XP=26O)1AFu)?|OCYHCJm*LP4*zJ8Raq1u z)9(U+oYRkni_C&!f4&%ORK?w$g6<;rT((@LunPCC_#2P zxJ&Q13mCI_U+H?IvV89Y)i_#NnNt!>xavHwF$|O zXuHG5oCo;G6F&W`KV4I0A-(zyjQ;ws!05mAr~eli{U77e_#bTiA4Hr~$mBnaBxQ^3 zlOJG&4aI|YIUi&Z#TBHjLS(GmY^z5R28NolKW$l^Ym#0I3|0lI-ggSR?CgqX8f;MBaPl&YzSG} z4(9gprQ%M^N3g+r;f^a0BNw0BQ9}e{Op$ssU!0cTdbP z1%BNUh*RkAe#+jya`#(*p*uQ|spESDMarSs8h3e`E#gtvYi=8d#ADvy9g>R@*^D~F z2t#h@kzA0JK)w;AMPg^lWi2XAU}jpiDF!akXK|rSi6}wmaK)KT*81I6M}f%l3XCMR z-&LC;?s53?Q?B;UuDeB{5^S+oOfSGE^CnkvgEc9^13~<4(iGap$VY8}3$6;-sL}t1 z4d0l&nxB@pZuYHH` z{ONm|SH}iy2^)Zg%Ou?*Q?I+u&ZmckE<;nVG0STB`M9GzLE5UAMeRQQJzJxXBBwA&_T6LHe4yGpP7i~lax~#Ub5BlJE zg>YF0Yn0Wcsv`EJIW^d7i>M?PO5_+)OxDS;9?zPfCH;#_rpR4-*9!|aogttErPHlR zUf2d~4Xa7AEaZSe)Mn9=Nd;=@JUDKUaJU-Rx~HXERZPZJTiBwHdXup>tP-Z$yw6H? z{D8e~w09((x@w&~)75oSpJ7o&u#DUKXAP}9afG;3qf=+XWeC!=Ip8PJvw~{@B3H)k zZr>U-w?x^Y3%$zAfoF_*V2Mlr?I=_C57F2k-rurm=_3`CHmW^yY`ye5aJG#E#oU&y z^R4vJ!2z7aF;V5BD1dbHn6(R25;-0cu1Cet+$J~Uw}=H_%79gf!-W2#1g=S`%zSN- zwVT1}5o>Hi-DpkU76(;YW&Y92O;@cEU^coXt>XfiRWI$}_*t&RQ_K?A8!$gpQKZe> z6VsBW458Q0>X1E#m*K&U%))^SmEntSPBAZb7VW{C@EA7Plo3r-`7EMb;;WeQn0bRTSxW7MTSYNoW=(qCsKsMVCbY?$#Z{|k#%NHM zA*6=sc(VKVE`UVqumIooHMGYRSh$SD{ErAy8%i_*n<=4ODdFErVql6WIx-X4fyaoz&jU+aYlbi=W`&5GJ~zS*@5IRv9cn<|il?|!d8>N94!OI0)aLF!Q0nlhtv zV$SFv61Ek9=p#mMT*~J{BfjK)?1ss~7B8LE@RPM6>=Q&sCt<9ZWOlek61x3T53zDy z_Ki;P_XP~dr)aCdrp;^Xx&4zy791bkXYcFE&ul#uoMVnctVZzl-Azp*+fw1N@S40^ zWBY6U4w+j|T8!q!)5)=7rk~;72u(J{qztk$Rb^WOCbU62Z^s|pn=)TqT4{gYcX?y1 z?|~>Cvir?R7Ga#&UI_thW{axhKZmGsOKK2*Z5|H*2nrEoD6q0cA?LAuQGqE#iVxT) zkKFW#vDut&E=}&^_xyn@nKhBk4S$!WNK~%$ z0c&2{SDdyuxlzV0ph!Peph$e2NH|n4;u};Z5-fDRQCkV`hd9~Qhw#l z5yeB&7zlX?y>QU?3e8P%Gzk1X934Q9LPIvcZi~Q>$tU#A^%^O!FsqRvO1M){#{wo# zBk9bs(!8G_zMYJ-^KkkOmXlld6&M}R+at4#TYfha^(?3_OqFsw=T6Gudap+sqFPF0 z*6D8MYBS6E;rkj8{7GbNPpnUPv9*l#u0T^M#yAbod>pw)srdC}u6;9n!}f|*m@!$~ z1aL-1&ei+i_Mkf0!?>5p@ss}z+(4GaIZ0Tu^mr{+M1{}bS8k3r~HKz!?C`p>TW)1H#Yg*vr z7Y{a{9Z}e1N<7QR%urOa_cLshyVKNaKNU@l7j~j>PeI7MIZZ|r0*YSjU6P_&ia|jH zDoChFYF-JCkoNDw*&*{QG3x+J%2L5_4`n1Tg9hatvloFoYL01#hFFj~!}MRSdgSSl z=m-yq{#uwWUIpuCs@%BEy5ob11|s~&TVX8~-XV)oMfeNdXD?Z9E10-tP#Krhiv$@dBpKj5J%t@Y2xI!*8s~Z z29}0zR`_9s&89Brq4Tru3F{G&uQu{ujBFqN`NY$Hb>qnXc(a!g%hbv!R@n6sNonM) zg649UVVIiIE)_J6eMZ?R^6HGdRMn-UD36*c8_Z2r&xc^Cs2p^v6x-_j{J)k91n!wt9I-~_PA$GNiLi=u7ixtk`YUQ4uIF+`SI~U z1J;MiD+DHLSA)nBsc8CJW1Z4F5uFXI0GzFHhs4egAoxF&>1&8*Nl_OA^!wW4GJCRO zwS%7>sOyj*5EN! zUpux=mBP|Q*_J!@%f6V&EZf{?`H}D&1^^@HO#Gta8P{W+FkdO5OW;fnD1|4&tlh3} z@YGnJ3d(Y0t#ep+bksNs#e?8*u-V=@#Dvz21#EB=jam5x3MtG&IuRHU$pr(K+Y-AX zn7FqKEk!?hw{HWBS~^ioY8Dbe(VtwFva+1h5$-}M9!~UYHGIL>zwFFN1`lcLe zwaMY%;tKHw`EL=C_^}jKY3YhWzg-&!anlG&@4E|`Vl}0q!EvCtT1I@}=Ug2;8OzB) zmllrTJ}RHtO2N@|-7)oaf*v0`{>2c|j?-t&WbDWOUDsBIUR24HnS0{I;>(%9+r)y* zg2K$nGPerx{E6HXH@h?eRQC~Y44A2^$`xKRwnOj_7pT5_!?K%>JT+F+ z6(@ZUF%FqvCBG2v8WL04A5>D=m|;&N?Hzcdj=|%{4JK2j_;hMKOfU}I+5PVH87xo# zc>v2%1gFE>V^6x3$7#ymLM62}*)(ex+`ImB7=eUwa2O&zcN_th9iPz)#fXNbq_VnK zg>+Fagfb53(>-Y^v23^|gST@kT%3pG*YUyrd-zn|F0Cr_;Qh)MO;mTE$%x&%B^Oc= zO-<|3$Nplt0sdxXQO`|RVIbVxm_^24G_6XuTxk&{Yyl+?OeXa-!t}8&fuTGLZpS|{?$S9qu^8TDrgtdOu`4*Sqx20lCJ(;z6u7&0EbrB@495}e zvjfw8yG7#Eo7QX+`k$3*tbTCwGm9LGOvTam&Kk&4&(T!!b0d-h(+s160p@Pn+_M|) zwasiA7r)El>t5DJfiBLb@2=gQDN0N*FfYuh&F<6BNcc)=oqju*S(+ucbzy4pyN1%s zgS@}T`xoCKJdeoM>hW-Zt9xSNRYI8RfX^{UPSJ}y8$_k~4-2G8KZDJQl``0lf>>)j z^q^y@`VIX~W%W-QAF*8U#?c|>tGQ{a09;)CL{-NfEv_2<$o(R8`V7xFRTl$)d~KX! zxG^v#xd(Z9R*`P* z8NwYSrl;qaYDzF0iB%{|A(v0($}TDr##;!y6paThkw{fnuKExakKusCdM>46hESJo z6Z4inrJpt`IzSB{l1R?`XS)o3@M9OZsiP&{y4g5QBH!U*Fvdd|9inn^a}Nz>2&)`? zh!|tcpGBMA4e|H2Y3)~7iyNUBsc|aN0$HM9Uc2MDIL(61;J!I)NmIwv>&&25`&+6M zq1}!I%Azc>=L(6nYlCWwU59Ea*szPa>sE|5)2pJsAnOmce3ZqxF(4^b@uZ6D1K#-5 zD6|eu@+l+j4}V7yxluQ@oX?sla^=5dw}yP&j6E+69hswg1L1c=)OyvZ7^wHQJl;ml z_2lX#$i;=Fs}vkh=ukc4y2Vj2Lu7vAHQ*E%@5?3`^a{BzDVU zF)O4|`;uuAO@)kfdwp~fqS#rR$4Oj@c*zBS`-fL6qu8<7qzl8rl--^kjiCV!(vbxC2vIdMo2I^X@+ID zcT&$52_`~JOBXh&mXX+ceO*m*0_=9ArqG>xjMR;+M=q{e-N#QEj-BCAzAVeGSrXNh zCV`uX4qS?7l$u+*J~5P?9xlU2%6rgo30lJ)cd|FHtEmloD@8tO@5y7N5t*NZN|hrm z*0FP5k0_1u5$>dp#I>8az>my1NoIAqBZ!Lx(!ohP^U@&Vmqd8 zH=75V+`}JpR;Wj8!j6BT1WSjMs>H+3_*52JYs(04P<@$3WEVZ7V%N-CLN$onNB~*- za-hT{!s~K{EUyaw7zDbp7n5T~SRV3$*>Zhpg-*51L=Zj|oeHx)1Mr4juj_5;_<5%8 ziMWWR&MhgdLq0$}U0q=ol1xb)TQBdcV!(3$iF4x~ue+F-gFAGMn^|`*YBjuP=jx!~ z06>UuQAq?Ix&zn0^To|<4!CSXZW7o6VrM}5dYxV+Q~8-h^Y9DzNs{5%+kyFy5cysy za}2EkZyRxQ^Rgq)T6r=({uw7y@%D4S?wd{Ck@D0(;mjg4NbY$Z$xd6rCGrNITO04Y zO%6aZ!9hMp%kU=V6dLc($d`AHMbf`&G9BXY%xr$$hovCbBj@|K2-4_HjW4Xn{knIL zaKV)PQkC?JIKYK?u)1`rzd)G(eO222!%q#U6QaT;SUl*MO9AvJ_$WC-@uTOjb58L_ zQo63V8+G)0D~=S&a%3>qqG`7N+Wfi$Logc=SXGBq3&TV|=!!;Nzi4VeqP9=hV>H5k ziX8p2v_i>9nc1rQm(7T8t#sTSGnI9T#Ms(_k_%sm3mT6gc=YrdUm@Ip6xRqL0H93*Yx0O!3Qw+_Y!81*n-ovS%iBlXx62TFNbk8K-j=LOV=1s zwc7i_TsS%sk!R7r81r4v*Ec`Rrl_m zr2$@wBrDGJ1`%wG6Ar259e%+MkZzK88-X>M^WgfA@HcWJmPUeFdO?d0>gvCTn0-ZWgb;$}~gdQiffS0?*jk$T`izb=V-&N#O_U4yp?Y!Mdlk09!o82t}+5dEvSj%vN5 zCBperFlf(sXr6C$n?zYvm=YYyz=~W1tkhvu1wODh>tKoBEiRB9*Py%96luTxm11-k?Q=g$c>y=q9%J< zVbw|kc=&DAiz8G*&G@8XlevEthbWV6a7nM1@VjKNkP|sl%x3(c9h#|9HIdVuC_??C z!MaVTrRI4=oMEugDa}D)#f1zPsr&vLR0Zy!7;QA4?x1w?=X%tH7o_(2z@8LjA`t^# zft3pe@**E=P;MFXEB+)Zh$?+;5%i6ECfT?A^~N`o&QHR5@V8a13HuA~omH+0(xm&s zJn#ru(@aCcl%uY66t2-NPi-*^o`hAyJ}I5kdqib+qh*CNP|jg>f!Wj#HJ<4r?4uCX zvkf`dDbhurH>#bk@3|Ap%0+kV-0PkcrZb0Q6)EJKBfaiae*!zLC7wkQ?cY#avSAHH z-b1`V^N9SgFL7-JrVQZS2rsHMA5v)j^@ga==T4XfE9yy6w7~pXILh8O)Le{Zg)9`|o`-$nca zc~hvlgOB$pGXop$oW3PzOuUbE^uRf@bo%^%%GEHQ}3uc0E<9SxbN+Fk6DEin>4 zHcD4f(K{ENOe$J0HJ#urqwE!{iYCcrgQT6kUmRQ&pZsx(U*x5m938GK3cceA-25P7 z?4_>Rtm;@LOJc>-Es0d2lZed7(#_R8eGm|eZ(xhjbvF{TQvs1jaS#K%R>_hqN0n}TZ* zkc089?X9=$pO*FdJ8a~1LwKU&Tl*+PUpFFBdK=aX&m5jxjDg5G1pXXNL&FXtQoDIi z%I2VE+_J15PN$4XB^X2Yje8=^qT3Q6Up)7auJ|SXIn8t2lJM#_5ql$SZ|nXfb&U<5 z+WD;cxsrkAy@tew0gl8PHWX0(qf>97u#=sJz7BD=`gp*W%GmlPa|+rCER@9rjcWg_ zl26OYrAyJyc>(x*jhp9DekXff;UF2NN;Ui}MJ?5ICzv@f9ALbJ?E#ZUr9Ic3 zzA*o$&I=Ta@JfZOEAMmeNUz9k93p!8X=>FBD$#aW*rJBSOJG_{E4u;M3A)vn3ZA*FCGn+Fg(4w7}cEUuvHYjNe3srT? zjGbTt%LY~=@?&|zrxYJ%v<6_xj4<+!VwleU+BF+z4)}b&?KFik zy?KZ%qJSTxm)WSC(-)vC z_LTIFihr!^y%i5PBEEPCOyW1(0O<=Ad}++TAQlUVUet+p^E3c}!Hm6Ker0kttjBIWHFAYVE28@r68QPb>)Vg<;d0ndg zIOg|&%Z^&B5koUj%;;F55>#Cd>y`X1^41GHDSIjVmR%4uBt$XKaBh6+p3un1m6DKK zM5nC$KuQFHa!O+A!tnBN$&WmSvCPz#nQaEXC!g(?sW+Y@AB1kdg2dM^(Gjmzs6*J zi>IYc&r4tXJ{{+;xx*UGux7GmUyf}GKo{&yc+i^CQk+fM5xwnR=XN< z!u~>Gl{|8NtTsKC_us}+!JbSFv?wd*)?I^VPt2vT`c;a6orPS2Qhe`>N1KB~dB}yP zspLQzZ>`?Hbq-7qJC#l@Vh{gOd0-=i*!QkM8LpL1X8-}g1mS#mh6v^#lwH+V0EAht zLRoZn@;eAS)m=80s0Jn#+sLq@zuIq|XFXByZxLIoN4=#LqQuVVkJJJoqdv}YdIi8` za&=Ppx)n$aP&MKW_^PY6l=m-iPXIGakyd*1%=})EsxHySwRk^AE?qcrR8hTjF`nFh z)+UT>wL0VXkVCY=24X|7B}!a=Gf)c2+1jXZ;lwogP%J5l_LHb4lWDj;(dv}Vr1IJ% zBzmFhafX~i#<1bqv&puIYKuHOPY|K%X&v{<{=yTL{$8uDcy(HHi}VDVjHC}Z7W0`b zEvA9p60jBWkkB5Rk#%5BJPS(P7jy(H&ZM=!PzvrzF1=cb@j0B{!WqXMl>4hvAUG#n zJd@sf-hvm66(tgSb~I9O>_*OH9ggr<9(jkPzpUP5U;9oi{-`RXFkT6&7UzshGl7YK z=w!GA{fajfE6<@$!92K|Md|hQp!i-X2J~nt=D;7#M2;}9l3LG<6`3C2w+L(}Swn*C-B*?`-k7j87(HI0e zOg>|2NSSo0G$Db|yJ=}l3XfUHc3P)1NIM4OhMgn9utTLY8mQE#BnS7N{&WXwxbPTC zj>^Vmu=6JO$5zNwB5NNSl0w;}jb@J-VA6wNi{X~PSBBYYx)&mpWiwGyMd~%>340*O<^m+;13xv+nsl@@4vWer8?fJpf?QLDsIAYG$AW; zLaEVbXdlU68j5l)of@<#27i#8e9acN)RqV5SD02bMKnOYW!RB{72(fvCCTBSVi?ru zbgDA#*GRW68N(c0E>5u>u(SP<+gV#x)7`Bp@SBKiVu<5JAQnY_TkLETuOirHXdSvS zvj3FIepQF6dAlF4aI!UHW_6)6yAM7CrBvn^#Qb^(|KMPUas1SycQijlWVnLIlvayxabGnXVuaQ^dHa@y9)=$QZH>SPegN=OO*~ zE)SFDbmX`%K>u)QKvO4)0Q6_1yp?lfgooarhtt<$z~YTO+(JVl(~ASc`owLsRkis`U_?MIJW!nR@Mo{TY+o9Pv7gjq0Br6 z69CC^k3Y>byZiTYSu$_l7lJPB2#srl$j1$McL;9;1JwOOnTj&h4}mWH-Vn?pBA#s3 zjm-omv~5W85u0g%GVKXOn)WQaVM*sXOrslhX;tKH6?3k};k`m#5;f?oYG{A|jfzVI zEawoElA5$S+%=j>B{ljl6OB6dMOtiz$z|zws<7A7tg64qMADNf&^>0E_v(v4Xo_qH zV^U-nQmvG1&4lmI`ITySApjtTHJlbWG-M3T*jAxeFp8eXd~QuT_;Rtxq6gbbb-=tw zoQ(PY91W&wSS2@?%S!N+c&XI*-Qe>8h;>EoRGL|8iL5JVmPFo`8mCcY@G7$%vVy7X z7@ReiXO;L?;tk6Mm3?VrP%a+9@9N45(_m|XD$^pZCLI=|=N&b3Eye{UTf~qseLt&P z!#sl$Vu>mfVC$4UM*S1iA&A8WT0&j2yWtx^d_y<4cNyNemon|ChjXI5IDRb_6+)L6 zHL>y7N+Zt&p4YiL#W9q4j^;U#_Uo|iALm532s#R|g|RtF1ga%u9(|3q*VEV07-Y_# z={jfTg|b)%84CRox5B4Px#rve>wV`e>F+Ihvw2o<_Q-Nv6Oskz6Xf0(P5Qe*HQ7l- zcH%D^p0}1DkU?Oh5Luxsh!wO zKUM!6-)%F>W(*eN%I<=x(m0rDftloG$@?ufi_0FJPvZ3#aSQ)qBP??BlZ)n3kR!u( ztnUxe)+T0*JsBGnx*NQaQ*rbN@u7$&a*QhLA>#~Ru<77+YbIJviqYiex1fq>1{FT# zFdi=DsQwOIHD+foydCEv&;U6m{f)}zJS3hga=b91my!N=YxAFN>}t3rbzl6j(22F3 zN=wsJ^$u!O$eS~g%{1`E%Z4(MfN(74t3fvCmpBFL^Zwb}W|;;%1`>f&|3*$y)Z>cJ zb4L4u3{QiD>q8`;X78t!poKbPNQ3F!N5@gjzIaM@VHUUjjLWq@kvi9sqbqS?nXGE8 z#+GiOoSb3agPl)kT>OYk63q+oSkS>R1&~Kn8mWrR@Ghg2kK(O=B0gr7cqQS&ZU#=n z!fuWk@yB<^!ZQXKgv|$6V&t7P%_Pw;Z6eX>n7u0VO2tT?Md1A_{XTzc4f!^fy@J`@ zL_xHu4pQ2%+0gi2MYpK?iQ^gAY+ZY~Gl4zpRA+4JCqhte=){_!sS#6~-(u2O33{G&qyu-3N|Q&_I& zrYu8ewgXs?(VGq;pSXyDqUfrqm8MV7=*kn-gajV?A&2rCKCU2b%V#8DjIS?*Vby zKbhSHwl(aey@M#B8n8X&2S?C9fc+T=k|2m>1p1jE^8a*p7GPC1+y5t}yFEv0biZjerCkVf)}=vc*AQeLaes5@b#F77Z6qAz%l-99zN7!krPb@WE@*haV*6;&%ac`t z$p+!J!?T5Q(0fA5a}OU8+PZ!Ndhf30kT((m^9FiJ79WS^vcFZ6gGuSj{S`e2Q%u8$ z*$=`FNUwnT3MQXg2wm@iypIy_wtTRvyLm345nt~Hjh{W&yk9bNXi)x$TYOmqRkBjR z62UrkX=#b5CsQ=dI{nd9hLOmmydWim_?39xb1J`JjsCP(>wNM~^8+bwt(VJK^`0=s z%97EYPT=bjs((ZFX-|N_y>DS zvWRyIuDcghz}MpyZE#*nQw|a4uW0zgqtA>*CLBdpjUhRD`mJFRa&;l=cRkT3S(l<+ zO8=_HSCLh~y|ftK(ajUECd|EE=Wy?Hb%c%#nHYPZLw9akcR7u!w5#-PioD>8RhE)< zt{&UjCzWN|o#^vd8j;6KXf=4}kMkCW| zVSxvE=u0vh*r$0-S(9P7Q5CW%^7bKVu=| zk>ZOJ}2*@xw z%?i%k;pi|RUQ44_+hrd+)y{B|7lfBZp}F!E)I)8)h6ld30f2zQD zTA+dMr02cDX+vCzfK9iwIK=x(6Jyzg^uR7;c;;@nWi3y`O@AqwhJ>;X- zN7gfZGgG5gwbGh~E(12E`qln~DWZnEFRDh%yxmP)2=<8>_4(`U0+5>T-4EU{^0T?< z`+eP>KTJFH+2mikxF_l^Z@%c<4BZl2RS?NPZ1r~7eLM)%xk}0y=Acd)Cm(z~Xvwb0 zQk7zx^wnc%U@M7vM_a$zg(1pPLqISuKU(`;+GHB;XjQ`ED5yW)tP!0z#M2FKs+Ds` z@d($Yzm}Bw#6VTT%Ge5*n?cNZ-1wB^I44Q442Ll-=xb?uqN`n``RUrAJG2xmJW}#I zW1SCEJv%R%*ur!4a{!F-lTBUWI$4=GO;;xgrKZ*Jp3sa<>ilJ{rnNT~(~B#*XEmiU z1~Ed`QBgYpk>YsHbLx#%E)o9--i+ZC9f^_7T3q*re!~_iq1d4WhP8%?V(#=QM(g^7 z>2+F74STNRx~BuypUTi!+)M{gS@jyMH($ZDu zKjsY7wy_tY=^3B$W08}!&<@2c!l~K6&#D)VB-K$kGlCyqCHZOrNP@szFIP8$SAP6l zAIjazY5FRXfEyma)Kg?SYc6gqIrvj&$otnW`!RzBpQi4fq)s=P5CdQP@)yndY7bUH zan{vp_Qu7}wY$KTn$j1%Y@h6=n?MZNqDJhm%WboRANR6CQby3{gRzTJfUkwKimRra z>v20v{=}dJ`%D)e01bVn*OnnAnvxkDMidvnnJEF&DTbM&P+`Ujq+6c9syhcdm!joG z*1W2nVX)Y4=7jc_kF3u24hP6*6e_ugdd-Zx2G;^;ugxy^C3B;tZE{9i)S#}n+Tm^Wl z^%KpO#g^>$))G%Ak1-6LUD#ZTRTn(7!9<4(>I$Q9zeW_j9T{_T6J6i{a*yI=rhgd@ z)gG{9+1{|l$zFGeY|`t&%G=$#LakN(kclKjR)UF-Ix%+c&+>+~j$d4Qmb}LruYMO@ z`qpSxlDi`75!wy{eqU`gG<%ZOL3iz#AK@!h!=>|j1B+Oe$GKu9eUZ!k_(1T+S7_kA zbJn;fO_sAts`Puo#$t6E;ze2?q_a>$w#+0nuk}*bYY8_IQmYk^aF^PtEnm9%vS?g- zl=f(*i$v;};DFLu)Ie}{;wBfYcRZ;#gqu}?q$J)G2lLswTD<(sxB!k1pp9in$Y8=k z^3JyAcETT9MmAB~bYMX>W~mpKeS-AdzQ{3eH)NL0Fva9G(r77Eq^5@T^jqfFHlZW6 zX`)orA@BS6J(?KBp+#ABTs)dY-6)A)m=B$=fl;)gp0w5h=kVgFEy%>zT==t#)Oswq zTr?{tmWGWFbDOksn&?;8ZO@~z1|4maoHqnx;)hZai1Oa97qKZ2`=>=Tqbi7E&k^Na zZ{=(CC~B6eo5t-^lBcfd9J7-)zKvBA>K}~;QMU(%+w1B)Tm0HTIfLh#lU;3Yn~+}d zUP0S|jo8kZ7+vu!d=$BZlVeRdZn#XTYejHx3KQ;O9%HU#dW(r^FcXBZC(y~Sm~%N} z2AJNk$S5a5XzSgPM7Rj`gO_&{#IQ+BaJI7%Cg(lRcrdBsB{DM zT8d*WSa9l7$|3s+xddzetVv2FvHpTmi>HO0ST5olCxQvl(GCf3Q9y&j7i|TuS52RC z$Mq$-RNqf4At8+FuTKP}#H=tDX#`r?5dsa5dEA@$R5+ZaAl)jTIpWtmtDot`nN#*n zhU~NvwXJ2@?Ng4=Ga)ngqKekQp9>riEd9DzgA}4BUwqIm0%Wss9jHUl$nKYqO;2N7 zknpSn9IQrcJR>i>8i4TbCiE{yOjELbLUDeF)~y3Xq^W(@CXkZSMd`R;HHADm=DLkJ zS;1I$?g$Acj(p>KT3D?`z_4LUo}Uvij?k=_H9S~+>bx^)AG{@fB`}K$xi6WJ!FPJGW zB~LoXg!SC`+S#|tF_WQeoMF^8u?W?f)9v=3VwpXM#@dD`br&6k3%WzaC(pjfR0`fM zChRRAn~rhB-s|T5e1XI1$7!j+-kyB4Yw?uPR@@9KfpTk%nATjRS13yeX_R>U?NRR* zYr(<$9=%ADVmjc*1V?@FRwNrtIjAjb6~xw zC-sWFLtc2tkj`HGvT-)9R$lY{zLj=HPa%BG;Eej@!{!SgZ7uQSkiTpuyam5P z5rGi-YQWO|GMX=FapkU`5NRBgpyZCbC47f9)TZ5%PIz1ivCfeoh~;Vbi@p|Pw7gM> zwb+um?aH84>hd{#m`B&9Hw?kAeS3;L=R7r;t*zfqC&7JCTJ}UUynqaE9fG)Oeo+9~ z<)#K&_ox+Nw&lB+9i|2E!p?w#If|`6#-*70{+ZT9cyNps75*mHJhbjb(M$RiL#Im7 zkt@=c&>5xhMt!=^u@mJ>AD$D_6u+1VyRkNNNm4B-5;&h9$MT0M8s71AN$h*tvfb!k&(H`x-=+RpQI>om@b>eBy%{M}3KN2#u_7ZsoV&Xy#uDxoRl2 zhZ9oKR?*q};PbY(m7gWgt{z{7YV^%w zc`Y^X^W2*`zFzR@pZ`FAYXD7ajJxrE>}I9XGO?tURZlH3Izhh)mjN#;L|i9=q<*Nz zeJ$l3es%o;Vkm2YSg0p_sEJfD;4905eJ~)3KL*>sr?_0fwyGKtmV*Mx?gOY(=^nPy z75*rmkv2($3TAtHYhv>G)jB4hBOwj?+DEI7B7nKguhhz2Yd1 z5R{LN%C|hj+rB0#%?eMKUp2KkGARiM^w%6HC3B_ajcD)SC*>BKm^LzSenJ0Ao&OwF zP*SjP9n;qLfKIW#zSsN6#KjQ=N9BF<<&EVWEqo{0Wy95oba_&mA2}DQZ?GFIAE4+$ zTSWyjBPuJ{I>+2{`XjGQUK|-8z?*tIei@>sC0eceal?yJ)H4CGLcpm&tzj$W8yN`# zWW`Z58t<@KB$*M=mUB3S1Ewuu;KvZt)Q44I^sc9(<6KD zz8jzDcL^6W2q>?&+~@GAhGm!bSVyKo4FcZIG@w+Qpt=z*Ug35;iTEV_r3KuuIY@AP z86i%AyiC(GJ?msLDzV2q&uEWf<036blx`(bK34rhL@TD$CD~KAPmc@j?tv4i(U$`9 zcWk#E6!Y?LEsmMJ0&nlU1XdZxd)a(3uMfNLXuUp;?^_>tzV(jaTa$0?-?6+ps6I8M z^B+WMTXsb|tcon?N_dCOn5B9n=!X7x%?0 zTWoPArre~5nAqwvGIZK;G@h1ctA0q9aR>+@?}8?$AnXuMICs=!+GRwXA9E?Tb*cs~c2&|aJbq|eJ7f#q| zoxW$gW$NCNCCs5dI)Z^%IkU1tA%66_qyJRWe0$h5=C+eor|YD9VtX=mo9i~)qd6;iM;BM3`Er9%Vbh*xkQP$9s^g?<6<&loxpnjh84ZhlM9LxMJBc zLXJ0K3!L}(&LVO@gM{JDV-#1QVN~`dv!T2 z2Qn;Li&$}sd(ekuw=gm4*!C?zfH%!{5U? zO_#Y7qV!K-j*(lr3xK97+d&CUgC{~Jh<6M)O$r&FwN{1 z20nbi=4jRBh^n!*wjSy8azByNjBI_hrIYM>2DjX@lKe#Cjb~HNQHwH_8rD&4I!0l; z_yD1aD4HlIRpaTe{;-Dp(o62$P92GK;Vp2_eF?x?niw86wX|gzR^&6S9>(;XlZu!P zg%R|xezBab&$a_p^tvy_W@JtUC?XN}cgE^{$r@Jj0O-eGw1y~*_g%tgOnARkghNuL z-{~{vK;QbpL8{T(kM6bO^)h}ux~es@-LTd;R=9)sxy<}5O;v>vrHj%91Z$l;<`Y(w zbdlOcHl_DeY2!3@#q;ILT9*;B7%PjE-TI@nj;lVk>o~L@x38XcbQ>sb4Q_ergjle2 z=1TP)RfEaI9>j4(%Pj#eMlOU;E^SAsx1HlY$8Ha+YL5x9-9of5SP~`Q!TTkHjuEe( z^@Be9fgW2rMRKH_{6?-ncAL`peXi#-uUai?&<79D<|qcq#{*VhfR0^Bu#$m}waU-a zf?oVYeZ&@3KR+@Wsj@7H(vYJuPF8)?g;g1qgAbPp;Ih|4hUftITYkRimR-QPGaWd7JcGhKSRpMGT&ZPF3KZi+UYK+VsaLymr zv>(Eeqzvw$N+M$wu# z>3e49=_k#bazg|41_rGVT0nT<(dcOP7(s1Ur0>eqr0e92dZHT8*{A<=?8f_)wMpo0 z{|aanXhtrN0z4$6y^uuRVHQ*`pV$MvaOW$EvoxJGG@+{pg z{B(^TDMUY~v>>L4)O#sr#wBegOIOE&*2iEbQW`BhEFF0u>@prRi!1xGtL|1g#KAS$ z2z`cSn6L;ja0_%*HV*2mK3AE;kjTw^YqTooD;21_$*D_&YbZt7kr0YIgDiIM+h3av zgXsG{{f0}-p6NrnC_K3|jZ}V2#|Q~}&q&yQGGhGuzGQpOxN92O13je4X(I|k==cr~ z){SHv(u91WcbB0wZRt+%i7bMlv;!;=?yyQRrb<4vGj{OKNm9nxng!4NsvZZwIjObb z@KC~nsdPY69@6BqZ5_xo2)t2U7f?&S-~;ZL?M-P+2NvUqJyv1rd0k&{^ggm|X#DvU zA1-EY8=0$XfC4GdfipYcF7$esav-K`gw%(SpA#*Orbj6niv@8kHC8^~J1)}`9(X#r zWe+dN@#5LahIxdUkkOvtdVCuX)hsK*ev-=yc~?~I&5QnUdA&FOi2aQH#JHqpMANea zI;p)iNmoZdlH(Y%N7`Q z$tJQ{7&y_+s7g)E&Jh({721M{ps2~O(9SBcraCmcZ0}dc5$rEJ!v9Pbl&6ubxH@S& ztYob|2_`2;c^Oa>H*AXv!H4p7jIMDi7;0~m>)a$fmh^tqSUKkGutJV0J%@winXVE} z1%Efz)uZZ}4@jH2eb^k(9K)`8{RrURx2bPm4BcAoetOQG1Yd9lGtN|#HSUjX16N>h zgp&z_RHqL2#CB%Ab+D{k$HbPfS>)o3Tge}(!1u2$?BrpEgXExq>_cGo??dcNzwR(V z`2az=)m9(}T9VsMQ)TcvTmoO*co=y?Ehmv68vM8`XAYc}We zjk&~={oCs$W&`ksP}g8;6e0#Qzfi1(I;sI<8?wAN#=S{q>b48Z8FtBqMe3Lo?t!EY z^itX@b~44Vwu5KIb~f1^NSYKTZoKLnZZe6uiSTR9JbuYG=>r+hd$|$O8?Z9?6eW!k zTvcHux%(;faiU}^r84lESQ4bMI=%MtQE>xOs(mCe>RrTGIvDfQnE0D5LQjK%wz@pq z{80dAMVzvl{BgUGwK)lIPb$1`LijJNSCwa+)WkhJcWqqlj9V`-C$fYU5EheRA zYafq_r_hB0^C}Z2UoB0XSs!8%AUq)yVUO) zwX6RI_&)zfJ?O}QN})B zszeLFN+26+QHH@RthaWS#8B>Gj$1KjY3qnj(efg95O48)}Hn;x28!H&jZ`_1+LeOo1{$L zw1a-o%V@mzgD3f2q79xeeEC1aKOyC7B61gS*S?_Zh`&^p>&?}@RO{q0!(DW^ec6;M zYT#36iu`t^u4YK394UnkPHrG6(vS#2#W7^a)DseTl(SK{_mRx$SSO(;R_bGn<;tZ{ z)`77$`ig8YMyqtHF!Oe^VW=Tk_L10)5Fg6Lmp5r4<(4)Vuimrx8er5B(n2pC(7r5? z#p<4o`2yc+!ZWADaFv&@35Yi_ve!%T@*JOz%$|SD0Vg&dWx_ie8OD<1#3l8(_F|Jo zCmXF1Uv%5xfF-Fk3?4k)4sbvl&!T!idJn0sbY#s!A+COh21I8hGu6fXK(MHhwc<^7 zjk#}tUy&wBpV8PzVY|f#+K#Y!YbCTm*g~AP zgs!E>RURoH8CYZ1E6;(H%K|7or+2N9^-bbqr-9b9nv)Xdd--LXSApu89O>+r&{j(e zsoCK3=YM5>U@;s1%m%t8n8Ez6Tl$-szkla^0A(mQvov>gGWtbU4d3`(1<+GX_por* zJEnKK!ZAfXWakj?oanK>w98Y9u$CH^O}GD3ny%d#s%lo*wAAtBn7P_V4@?f6B`EFdP27|nUbv{J6fxz z&di#|ozz#*%c7NKR-|Rr$zJ`G^W7UZb$KrG$#u0iQ!4Pom1;dBDrR`K5>p%fuIim| z)uO7-JkL@}EF$p2sMc%(@TkgyPCk7K`eakofj`y_h6>Tv{FFOv?|n8K1nWY~c$J7O zo$OnJ8VwVPt8`m#*V2+6*PL2&p-b36MazIZ^`hSGmUdct9ltF~lGm8yY_CPrcVPqF zbm=0sw{Pc%=v4NPkOWx#dk#Lxd4?Z0s9pr?U_k))RlmZg8}zO3szcme$P5m32;ToK?74f|_(j%4_CBhdvdOZ zAAS*wBz1AnzmDxfU@^OsTn#5a;%Jrku_al3e{

1bvi{DS7E@q1{$_8->K{_OWv2 zCZTgG2Pr3n8|ec9kIu&uC|d?k4-cQ4#}Z`qDX5Y2mhC(jR1Ms;UG4Ho$DE|+SeJ@{ zJQQhAXj|<)*t3KiOWTuh{Wd^mS{u{&ERV)OpZwiQ%#1->r9p zSK_^*U~=?ywH~4IUxb}{0J!SmL!z2Tzq_PpetoC^_az1JFg0=gMcQADuOP%3=H1hH zH_=dG(PD;d*037Ov5G1924U#Zns?~fs+eh1%-bWqa%ssm3=nio1r3J<4G0IBETtr? zycs~0JIOn;MecYG=~OQsYHIrf?~A5>_ob%8+uOrVA+VCJw}{lygrBBdY1k<8B^wf6 zl|<%N$7)fOZX$%y>4ueco_Gb1H@B%XrKVwrn6hUOecnc^PU0rFuCB5=*2;|u-`o(@ zL*tr4bnQzXYLc4XqFbv5sK0}A)`}`8iM8ehtj#Oc5DrE;0VxbPmL@BUa_BQwa$EW~sU#-LP0?sGmqfUGhGWcciGZ*4(}u3z=@b>Ow9DQe7lcO3K}BG3j(t& zH10>sK!&4Q5-=gN@Nxj6{|*nuyqw7KZJ1?p)NUJ?U0bOigGdsOk}Iz&9PmN_5=W*Z9M zy^pA`&dX0oo6?CSuhE~(pYbLuTPp1a1Fa@e3Lu&mmgd$;D}&g-i=D-{sv?J9kIr9r zrX&Z)aFGK^kNY{LxrotP0}k*;uN12i_2a_JJhKwh zBt{D-JRxC$8U+-`u1xD>gJ^H4lbW;7spI-=H506i=ncdK;xq*L6f7jVz$XGMg5aQk zHRJY&$@g}i_SP##iC?lR?ltnWUTT-UDlq(*BTQaYNkg zNG#sNoo{WmP+Vl}U~?+T?g25b$E-7iwhu=VVgw3JdFXm~ba+LC4p>CP3~rNTiNBl7 zL{RfLLepNPEtZj}yL_#R{(^MqIlG)c0Va}>U|9Pl&B_3tV;Ps{r)WqBznD7FcTlP4 z`JQe2DvGhmeeHGGX39zGyOOxZ3tq~Dft(BQ;mDXwwJi?sBtxo$Gf1SS2w*eQ0p&RVMNVi@d zY8v4J0(n}%6*Rw(g~l@sUuxpiJ*Y}7TzBQyU+>-qWm*InUeGt@)T9g^0J#z4){Lw* zT;69if~U9DXBR9fgVPlYy7aDhJU)gDC?_GHQtwa6QXNaah7-CzA|Fx-lH7d@N9>38 zX(F&fd3w7AkZ+ha8-gKfX%@_~<#HDs?kBg5zW>V3%Xw5jwPs6uni{7r zd`EfPYrA*SU;xDtm@E>5TrJKlg5o=h;NSXk)pt4K)GbpP0xkUg>2o|oG=`UnX7^Un zb&@8d6Fj1cBWW^c(K#Csc8xEBa4KfHY>8Lp^77-lhzgWr9kR9_p+g|-9r?VSv?qA%^1O;cqgke)%AqHlR$B{!Y1Mq zj|)Ecg?{_!>kGDAwGa7%cwSUb{BcayJihkv$}ql+yu=O}jVvAFdC{Hjh$4}u+$mx% z5V$sUiGCX%D3A>bKwY8HR)Gv*lisI4q^3vJ*nDwj|mtr!0r!~+Qoe2cw^jPCXkT7tI*01|w@ z&gPC`?O1w7hQ%=&bcHi7(fqhY3${~JepA7y@^aLwHpew^Yk$;R4v{ASHjXjXtaTc_ zuz5*nXB&PrcyWx#gQ%?HyxawmS+Wu(7ssvB1UMh!1$to&o(mv_f=9~!9@VsJCGxpu z`>g5Sp=xDhpsiCy^y>=fI0DON$&pb7o7^d{@@&hj3!6PUd=vA;G;#7&8ChamsE{`^ zY8pDra8Jntp62Ivi)Y`*XbpM60s06v@Rz^-g)TW_F@B!~y7!4AJ>37mAuz!(!C+xQ zSR61?u!{N|qHWOeR%$RXRL~vpN0SGri7-klNHEJuivbi=0qSbdV4&ghf4i|7?$>z( zI{qH?i}`~a7GyB6|8pZRq982+P*r1+m-t&(%U5#ZWFQd-(CXKLHeN@y(c z;wqq1hzE@q1b$GG0VQ_)`{MeylBlVfy%UHR=;Z98>T3M&;{0i?+0T-Bck?I)AUQrz zeF**_iGu$JlCpLnFv`D9?q6R51jKPM{Rd6!0FF#KP=O|b3iQX*TqXSjO?gXaXAmLr zU#g&%@+XpjVArlGkfaPKk^PUSnMLsjlK<9nH*zxl^V2-jGC$4+HGE%?F3%4|y9>HN z|FJgz*HW$VwU8$RNtuBf(2vdZhW3x;R6%eoJM(|2zvKebxCh$s5J-*fhZ75B_yeUs zFTrToFiB^SNH?gV2>l?G&h!UD>UP%uKh1L;Er59!q&NoZRe$VEf?5Ar^&iUad&2gQ z&WE`E%lTg=_3XQT@gJOjkAi-Hbbqrl{(pA<>_GH4O8+xI^=IAhS#v+$vmgOK=>C!~_xFg-pLM>6kUfy=zL|u~KkNJ< z$L?p*?;%(Ze6w%%M(zjE|4dH&5$)_}mG3z{KUQ6s!Y@_+kInPH;kAC&{T^5HKmqz@ z@+!aA{YNIy&r;uKTz=r6e6v>d-%9<%_4R!+-iN^8H#0N(rQbiu-u&}-|2`q@k1agM zdHkW_1&%VDD_|I;NpK*OZfAjAb z`Ttl8km0{|{F`kWKWltH$^Ech;G2y`{7&N^%H;d0$cGv7Z^oJNOSiwAFaP<=em}wX z<8AA6<}bbeZc_7S=ii6PALi)3nOXL)o&Uj%-OnQ52M&L%(%ZaWiu^(R{b!Bu2WJl< h$Zw`p^gE5e2}ml*LW4$nU|{5+pXG<~Ugg7I{||-5t(pJ; literal 0 HcmV?d00001