From 608b0713434a1978443b4788bc8f745a9f91db86 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Tue, 20 May 2025 16:57:34 +0300 Subject: [PATCH 01/23] SinkWriter implementation of String data type --- .../convertor/ClickHouseConvertor.java | 60 ++++++++++ .../clickhouse/data/ClickHousePayload.java | 18 +++ .../clickhouse/sink/ClickHouseAsyncSink.java | 80 +++++++++++++ .../sink/ClickHouseAsyncSinkSerializer.java | 41 +++++++ .../sink/ClickHouseAsyncWriter.java | 105 ++++++++++++++++++ .../sink/ClickHouseClientConfig.java | 45 ++++++++ .../clickhouse/sink/ClickHouseSinkTests.java | 102 +++++++++++++++++ .../connector/test/DummyFlinkClusterTest.java | 2 +- .../connector/test/FlinkClusterTests.java | 15 +++ .../clickhouse/ClickHouseServerForTests.java | 14 ++- .../resources/epidemiology_top_10000.csv.gz | Bin 0 -> 84696 bytes 11 files changed, 480 insertions(+), 2 deletions(-) create mode 100644 flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java create mode 100644 flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/data/ClickHousePayload.java create mode 100644 flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSink.java create mode 100644 flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSinkSerializer.java create mode 100644 flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java create mode 100644 flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java create mode 100644 flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java create mode 100644 flink-connector-clickhouse-base/src/test/resources/epidemiology_top_10000.csv.gz diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java new file mode 100644 index 0000000..14b644d --- /dev/null +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java @@ -0,0 +1,60 @@ +package org.apache.flink.connector.clickhouse.convertor; + +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.WriterInitContext; +import org.apache.flink.connector.base.sink.writer.ElementConverter; +import org.apache.flink.connector.clickhouse.data.ClickHousePayload; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ClickHouseConvertor implements ElementConverter { + private static final Logger LOG = LoggerFactory.getLogger(ClickHouseConvertor.class); + + enum Types { + STRING, + POJO, + } + private final Types type; + + public ClickHouseConvertor(Class clazz) { + if (clazz == null) { + throw new IllegalArgumentException("clazz must not be not null"); + } + if (clazz == String.class) { + type = Types.STRING; + + } else { + type = Types.POJO; + // lets register it + + } + } + + @Override + public ClickHousePayload apply( InputT o, SinkWriter.Context context) { + if (o == null) { + // we need to skip it + return null; + } + // + if (o instanceof String && type == Types.STRING) { + String payload = o.toString(); + if (payload.isEmpty()) { + return new ClickHousePayload(null); + } + if (payload.endsWith("\n")) + return new ClickHousePayload(payload.getBytes()); + return new ClickHousePayload((payload + "\n").getBytes()); + } + if (type == Types.POJO) { + // TODO Convert to byte stream + return null; + } + throw new IllegalArgumentException("unable to convert " + o + " to " + type); + } + + @Override + public void open(WriterInitContext context) { + ElementConverter.super.open(context); + } +} diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/data/ClickHousePayload.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/data/ClickHousePayload.java new file mode 100644 index 0000000..c35e211 --- /dev/null +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/data/ClickHousePayload.java @@ -0,0 +1,18 @@ +package org.apache.flink.connector.clickhouse.data; + +import org.apache.flink.connector.base.sink.writer.AsyncSinkWriterStateSerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; + +public class ClickHousePayload implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(ClickHousePayload.class); + + private final byte[] payload; + public ClickHousePayload(byte[] payload) { + this.payload = payload; + } + public byte[] getPayload() { return payload; } + public int getPayloadLength() { return payload.length; } +} diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSink.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSink.java new file mode 100644 index 0000000..20caa44 --- /dev/null +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSink.java @@ -0,0 +1,80 @@ +package org.apache.flink.connector.clickhouse.sink; + + +import com.clickhouse.data.ClickHouseFormat; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.StatefulSinkWriter; +import org.apache.flink.api.connector.sink2.WriterInitContext; +import org.apache.flink.connector.base.sink.AsyncSinkBase; +import org.apache.flink.connector.base.sink.writer.BufferedRequestState; +import org.apache.flink.connector.base.sink.writer.ElementConverter; +import org.apache.flink.connector.clickhouse.data.ClickHousePayload; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; + +public class ClickHouseAsyncSink extends AsyncSinkBase { + private static final Logger LOG = LoggerFactory.getLogger(ClickHouseAsyncSink.class); + + protected ClickHouseClientConfig clickHouseClientConfig; + protected ClickHouseFormat clickHouseFormat = null; + + protected ClickHouseAsyncSink( + ElementConverter converter, + int maxBatchSize, + int maxInFlightRequests, + int maxBufferedRequests, + long maxBatchSizeInBytes, + long maxTimeInBufferMS, + long maxRecordSizeInByte, + ClickHouseClientConfig clickHouseClientConfig + ) { + super(converter, + maxBatchSize, + maxInFlightRequests, + maxBufferedRequests, + maxBatchSizeInBytes, + maxTimeInBufferMS, + maxRecordSizeInByte); + + this.clickHouseClientConfig = clickHouseClientConfig; + } + + public void setClickHouseFormat(ClickHouseFormat clickHouseFormat) { + this.clickHouseFormat = clickHouseFormat; + } + + public ClickHouseFormat getClickHouseFormat() { return this.clickHouseFormat; } + + @Override + public SinkWriter createWriter(WriterInitContext writerInitContext) throws IOException { + return restoreWriter(writerInitContext, Collections.emptyList()); + } + + @Override + public StatefulSinkWriter> restoreWriter(WriterInitContext writerInitContext, Collection> collection) throws IOException { + return new ClickHouseAsyncWriter<>( + getElementConverter(), + writerInitContext, + getMaxBatchSize(), + getMaxInFlightRequests(), + getMaxBufferedRequests(), + getMaxBatchSizeInBytes(), + getMaxTimeInBufferMS(), + getMaxRecordSizeInBytes(), + clickHouseClientConfig, + clickHouseFormat, + collection + ); + } + + @Override + public SimpleVersionedSerializer> getWriterStateSerializer() { + return new ClickHouseAsyncSinkSerializer(); + } +} + diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSinkSerializer.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSinkSerializer.java new file mode 100644 index 0000000..f7db1d5 --- /dev/null +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSinkSerializer.java @@ -0,0 +1,41 @@ +package org.apache.flink.connector.clickhouse.sink; + +import org.apache.flink.connector.base.sink.writer.AsyncSinkWriterStateSerializer; +import org.apache.flink.connector.clickhouse.data.ClickHousePayload; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +public class ClickHouseAsyncSinkSerializer extends AsyncSinkWriterStateSerializer { + private static final Logger LOG = LoggerFactory.getLogger(AsyncSinkWriterStateSerializer.class); + + @Override + protected void serializeRequestToStream(ClickHousePayload clickHousePayload, DataOutputStream dataOutputStream) throws IOException { + byte[] bytes = clickHousePayload.getPayload(); + dataOutputStream.writeInt(bytes.length); + dataOutputStream.write(bytes); + } + + private ClickHousePayload deserializeV1(DataInputStream dataInputStream) throws IOException { + int len = dataInputStream.readInt(); + byte[] payload = dataInputStream.readNBytes(len); + return new ClickHousePayload(payload); + } + + @Override + protected ClickHousePayload deserializeRequestFromStream(long version, DataInputStream dataInputStream) throws IOException { + if (version == 1) { + return deserializeV1(dataInputStream); + } else { + throw new IOException("Unsupported version: " + version); + } + } + + @Override + public int getVersion() { + return 1; + } +} diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java new file mode 100644 index 0000000..7d9c6f4 --- /dev/null +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java @@ -0,0 +1,105 @@ +package org.apache.flink.connector.clickhouse.sink; + +import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.ClientConfigProperties; +import com.clickhouse.client.api.insert.InsertResponse; +import com.clickhouse.client.api.insert.InsertSettings; +import com.clickhouse.data.ClickHouseFormat; +import org.apache.flink.api.connector.sink2.WriterInitContext; +import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter; +import org.apache.flink.connector.base.sink.writer.BufferedRequestState; +import org.apache.flink.connector.base.sink.writer.ElementConverter; +import org.apache.flink.connector.base.sink.writer.ResultHandler; +import org.apache.flink.connector.base.sink.writer.config.AsyncSinkWriterConfiguration; +import org.apache.flink.connector.clickhouse.data.ClickHousePayload; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; + +public class ClickHouseAsyncWriter extends AsyncSinkWriter { + private static final Logger LOG = LoggerFactory.getLogger(ClickHouseAsyncWriter.class); + + private final ClickHouseClientConfig clickHouseClientConfig; + private ClickHouseFormat clickHouseFormat = null; + + public ClickHouseAsyncWriter(ElementConverter elementConverter, + WriterInitContext context, + int maxBatchSize, + int maxInFlightRequests, + int maxBufferedRequests, + long maxBatchSizeInBytes, + long maxTimeInBufferMS, + long maxRecordSizeInBytes, + ClickHouseClientConfig clickHouseClientConfig, + ClickHouseFormat clickHouseFormat, + Collection> state) { + super(elementConverter, + context, + AsyncSinkWriterConfiguration.builder() + .setMaxBatchSize(maxBatchSize) + .setMaxBatchSizeInBytes(maxBatchSizeInBytes) + .setMaxInFlightRequests(maxInFlightRequests) + .setMaxBufferedRequests(maxBufferedRequests) + .setMaxTimeInBufferMS(maxTimeInBufferMS) + .setMaxRecordSizeInBytes(maxRecordSizeInBytes) + .build(), + state); + this.clickHouseClientConfig = clickHouseClientConfig; + this.clickHouseFormat = clickHouseFormat; + } + + @Override + protected long getSizeInBytes(ClickHousePayload clickHousePayload) { + return clickHousePayload.getPayloadLength(); + } + + @Override + protected void submitRequestEntries(List requestEntries, ResultHandler resultHandler) { + LOG.info("Submitting request entries..."); + System.out.println("Submitting request entries..."); + AtomicInteger totalSizeSend = new AtomicInteger(); + Client chClient = this.clickHouseClientConfig.createClient(); + String tableName = clickHouseClientConfig.getTableName(); + // TODO: get from constructor or ClickHousePayload need to think what is the best way + ClickHouseFormat format = null; + if (clickHouseFormat == null) { + // this not define lets try to get it from ClickHousePayload in case of POJO can be RowBinary or RowBinaryWithDefaults + } else { + format = clickHouseFormat; + } + try { + CompletableFuture response = chClient.insert(tableName, out -> { + for (ClickHousePayload requestEntry : requestEntries) { + byte[] payload = requestEntry.getPayload(); + totalSizeSend.addAndGet(payload.length); + out.write(payload); + } + LOG.info("Data that will be send to ClickHouse in bytes {} and the amount of records {}.", totalSizeSend.get(), requestEntries.size()); + out.close(); + + // .setOption(ClientConfigProperties.ASYNC_OPERATIONS.getKey(), "true").serverSetting(ServerSettings.WAIT_END_OF_QUERY, "0") + }, format, new InsertSettings().setOption(ClientConfigProperties.ASYNC_OPERATIONS.getKey(), "true")); + response.whenComplete((insertResponse, throwable) -> { + if (throwable != null) { + System.out.println(throwable.getMessage()); + } else { + handleSuccessfulRequest(resultHandler, insertResponse); + } + }).join(); + } catch (Exception e) { + LOG.error("Error: ", e); + } + LOG.info("Finished submitting request entries."); + } + + private void handleSuccessfulRequest( + ResultHandler resultHandler, InsertResponse response) { + resultHandler.complete(); + LOG.info("Successfully completed." + response.getWrittenRows()); + LOG.info("Successfully completed." + response.getServerTime()); + } +} diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java new file mode 100644 index 0000000..852e9a9 --- /dev/null +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java @@ -0,0 +1,45 @@ +package org.apache.flink.connector.clickhouse.sink; + +import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.ClientConfigProperties; +import org.apache.flink.connector.clickhouse.data.ClickHousePayload; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; + +public class ClickHouseClientConfig implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(ClickHouseClientConfig.class); + + private final String url; + private final String username; + private final String password; + private final String database; + private final String tableName; + + public ClickHouseClientConfig(String url, String username, String password, String database, String tableName) { + this.url = url; + this.username = username; + this.password = password; + this.database = database; + this.tableName = tableName; + } + + public Client createClient(String database) { + return new Client.Builder() + .addEndpoint(url) + .setUsername(username) + .setPassword(password) + .setDefaultDatabase(database) + .setOption(ClientConfigProperties.ASYNC_OPERATIONS.getKey(), "true") + .build(); + } + + public Client createClient() { + return createClient(this.database); + } + + public String getTableName() { return tableName; } + + +} diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java new file mode 100644 index 0000000..2e375ed --- /dev/null +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java @@ -0,0 +1,102 @@ +package org.apache.flink.connector.clickhouse.sink; + +import com.clickhouse.data.ClickHouseFormat; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.connector.base.sink.writer.ElementConverter; +import org.apache.flink.connector.clickhouse.convertor.ClickHouseConvertor; +import org.apache.flink.connector.clickhouse.data.ClickHousePayload; +import org.apache.flink.connector.test.FlinkClusterTests; +import org.apache.flink.connector.test.embedded.clickhouse.ClickHouseServerForTests; +import org.apache.flink.connector.test.embedded.flink.EmbeddedFlinkClusterForTests; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.connector.file.src.FileSource; +import org.apache.flink.connector.file.src.reader.TextLineInputFormat; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.net.URL; +import java.nio.file.Paths; +import java.util.concurrent.ExecutionException; + +public class ClickHouseSinkTests extends FlinkClusterTests { + + static final int EXPECTED_ROWS = 10000; + + private int executeJob(StreamExecutionEnvironment env, String tableName) throws Exception { + JobClient jobClient = env.executeAsync("Read GZipped CSV with FileSource"); + int rows = 0; + int iterations = 0; + while (iterations < 10) { + Thread.sleep(1000); + iterations++; + rows = ClickHouseServerForTests.countRows(tableName); + System.out.println("Rows: " + rows); + if (rows == EXPECTED_ROWS) + break; + } + // cancel job + jobClient.cancel(); + return rows; + } + + @Test + void CSVDataTest() throws Exception { + //String path = ClickHouseSinkTests.class.getClassLoader().getResource(".").toString(); + String tableName = "csv_covid"; + String dropTable = "drop table if exists " + tableName; + ClickHouseServerForTests.executeSql(dropTable); + // create table + String tableSql = "CREATE TABLE " + tableName + " (" + + "date Date," + + "location_key LowCardinality(String)," + + "new_confirmed Int32," + + "new_deceased Int32," + + "new_recovered Int32," + + "new_tested Int32," + + "cumulative_confirmed Int32," + + "cumulative_deceased Int32," + + "cumulative_recovered Int32," + + "cumulative_tested Int32" + + ") " + + "ENGINE = MergeTree " + + "ORDER BY (location_key, date); "; + ClickHouseServerForTests.executeSql(tableSql); + final StreamExecutionEnvironment env = EmbeddedFlinkClusterForTests.getMiniCluster().getTestStreamEnvironment(); + env.setParallelism(1); + + ClickHouseClientConfig clickHouseClientConfig = new ClickHouseClientConfig(getServerURL(), getUsername(), getPassword(), getDatabase(), tableName); + ElementConverter convertorString = new ClickHouseConvertor<>(String.class); + // create sink + ClickHouseAsyncSink csvSink = new ClickHouseAsyncSink<>( + convertorString, + 5000, + 2, + 20000, + 1024 * 1024, + 5 * 1000, + 1000, + clickHouseClientConfig + ); + // in case of just want to forward our data use the appropriate ClickHouse format + csvSink.setClickHouseFormat(ClickHouseFormat.CSV); + + Path filePath = new Path("./src/test/resources/epidemiology_top_10000.csv.gz"); + + FileSource source = FileSource + .forRecordStreamFormat(new TextLineInputFormat(), filePath) + .build(); + + DataStreamSource lines = env.fromSource( + source, + WatermarkStrategy.noWatermarks(), + "GzipCsvSource" + ); + lines.sinkTo(csvSink); + int rows = executeJob(env, tableName); + Assertions.assertEquals(EXPECTED_ROWS, rows); + } +} diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/DummyFlinkClusterTest.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/DummyFlinkClusterTest.java index e958437..6067a2b 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/DummyFlinkClusterTest.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/DummyFlinkClusterTest.java @@ -52,7 +52,7 @@ void testDummyFlinkCluster() throws Exception { @Test void testClickHouse() throws ExecutionException, InterruptedException { String tableName = "clickhouse_test"; - String createTableSQl = String.format("CREATE TABLE `%s`.`%s` (order_id UInt64) ENGINE = MergeTree ORDER BY tuple(order_id);", ClickHouseServerForTests.getDataBase(), tableName); + String createTableSQl = String.format("CREATE TABLE `%s`.`%s` (order_id UInt64) ENGINE = MergeTree ORDER BY tuple(order_id);", getDatabase(), tableName); ClickHouseServerForTests.executeSql(createTableSQl); int rows = ClickHouseServerForTests.countRows(tableName); Assertions.assertEquals(0, rows); diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/FlinkClusterTests.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/FlinkClusterTests.java index a92b6f1..ca3bd36 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/FlinkClusterTests.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/FlinkClusterTests.java @@ -18,4 +18,19 @@ static void tearDown() { ClickHouseServerForTests.tearDown(); } + public String getServerURL() { + return ClickHouseServerForTests.getURL(); + } + + public String getUsername() { + return ClickHouseServerForTests.getUsername(); + } + + public String getPassword() { + return ClickHouseServerForTests.getPassword(); + } + + public String getDatabase() { + return ClickHouseServerForTests.getDatabase(); + } } diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/embedded/clickhouse/ClickHouseServerForTests.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/embedded/clickhouse/ClickHouseServerForTests.java index 532dfb1..50d5fa8 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/embedded/clickhouse/ClickHouseServerForTests.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/embedded/clickhouse/ClickHouseServerForTests.java @@ -75,7 +75,19 @@ public static void tearDown() { } } - public static String getDataBase() { return database; } + public static String getDatabase() { return database; } + + public static String getHost() { return host; } + public static int getPort() { return port; } + public static String getUsername() { return username; } + public static String getPassword() { return password; } + public static String getURL() { + if (isCloud) { + return "https://" + host + ":" + port + "/"; + } else { + return "http://" + host + ":" + port + "/"; + } + } public static void executeSql(String sql) throws ExecutionException, InterruptedException { Client client = ClickHouseTestHelpers.getClient(host, port, isSSL, username, password); diff --git a/flink-connector-clickhouse-base/src/test/resources/epidemiology_top_10000.csv.gz b/flink-connector-clickhouse-base/src/test/resources/epidemiology_top_10000.csv.gz new file mode 100644 index 0000000000000000000000000000000000000000..585800d690d60df3ddfa1bd4f74eb9527b057794 GIT binary patch literal 84696 zcmb@tWpHD`)-5KPnVFfHnVH!hGcz-dna1ofGc&V2W@Z}OV`gS%itqjMl2qzdenr() zs+L-P*518(t$oxIVH6aUsTrLm0MyLU%GAux%E8vb!pp$b!O?)3>HEQ8;^Gbn`MFj@ z))9H(BYUK1F3Zs^*s{>^^_7TV5+56Y>y!Lr zEBE8>>m&7?_(edi8*%YU^x{qFOMtB#aq3F+>`X~vE4*8k)W1~XQz7O(8|Q_W{no>J z|7u}$t307cRkZv6s}=f|O1wzGzxYM$zY+hw3YY38B>Dfn6G<38ng5OjBkPBdd?otu zrt~JT(1RFrC3^QZ`^tkBH{e3$Pxw1i=@tuS!4u&-z9I>XH?5y=zX`@kvFOg2v8Zlw zNTp?twk_j?QOHpb9QXySjzs(!O$3U!w~W|srOV@l|CT&kI>ageE>$28KoS0>9;>V( zRVt5i;*eEHrgAsePEc5+f~?SfjCRH>!*bGay`gevASfYC#gC1Lf8y}{uG$FVso@Am zsB(wbed^0b@xVya?!_Hmwcl=1_p{?7A~ort(H`sXuUZW~9l}u~;J%wbH0`&X1M_lVvJvSerkZ&eOgx{Y=d@ z6MW$=Fdd5z11xbn=d2?Of1Jxv7!YK+5xdN(n&*kf^EHE;>QGf~5(*_bO_ z`8f`>TJY6*IYe7a@z$&?78QnPH@u>!>sCaKCZrS55yOIjh%C( z^uxlx>3HDg+4G<+a=?pevi@Mkxf($%r0Di_<(DL_{!`^VW?7QW;*jemb4yKlGfj$6B*ilNA*DJ(Ha_5jbWrqIPR`p6MckbmBV4dslV=Zjm$ zSe>vsZf*~$Tk@`2S%DpopfI6+v#@5>#g%`_2e1FhvZFR8S-%mcN&6&}_wu3UeVvRq z9qOaw)-?OOGu{q6A7}YA_lXCOGeWb+jl;I0{?CO?={vQsTh$lb ze=cMYT9=wY)_!7Tf8pr`&MKM5D)SbuB#hW_tqGXM((sQn(-@_i`^7?Z>uisIqn>+s zJs_GDh3VzP?Xj#J&6?`0tV`OYN&stdF-_9vSy$Xu(Vz~D5` z{f^?m$JT&f4ZfipsAX{rdL9WnryLdqdg)DIQ3}DHGPfVXQB$ox;miv=LEH7bJ?x*ni=jWNW)QE59N*h)VC zL39#QQQ3zF^uONN^|%1Sy`52ioEFk6m1^n%L7>!O_lUZP;Bd^ifc~O&Z<{tfdyRAG zTh}5pfuJ#@KVy)NTWltYeUUx?(@`5X!x(VbhOuDnbiS_VdihQ{b6l#B5a8~=#?f4S zjb$K;i5D!5VZw0W^t!>6K>OCfWTtJWkWA=C48z%@stR$2Dr+h?zUyFBd+k zS;YZ&XmbBE+!t|*-}TR;d%eGQSuCzj!W!nUaC8XI-X~*Goz1#uM$FF*^BF#App0Wp zvP^9$^j`mFUNY+k-{Uad4f72bN^s}d^9{#fSH_(IN7}T>?^I52W67PTen_+D^ce4e zd?r^-ctx+t&fVWUWQE!s64(<}r11}vqK6Afaukq2;OWsQA%{x5w;mxGmrt z&ACMzJ9Ce7)#rz=TKa}XxI=-hO3pUC7@b3XUW|Wbt&m`IFy^zZGEC^jM3=9~Ng| zwIh;U+&x@4|29x}qA;D%IQ;-dl)QA^nz51}F?2KV@PYIA)`{z_DUg|+;&a6=WC#9B z32D#;ug93;;8Eg?+B-Ux2(QFX;>#&|O}^=8k9QT}9$CR9(!wk_zuSQcAJWbIH57^` z`2kz>k;+iaxUb}`Ibw#gU-h%`J2vn83jVL+8QMw@H6IhMO$E5-jE zlM%J4E;syqc?-=$vbL(kaT7|)gFV&cN=GetH9hBKu~0>fIMnPEjv#o=S_RI%uO_QPk8v8Twgmpi1?VbmaMKfgNy@vZjVj?WjA;kmhr-bmZjK|mR#a)@l; zArvYXWZU6ubzgF)e$PT2d1{V666ta*PYLG*s`0BYe1Qm|zfhIq7R{iquxD8_VLjZ^ zKU5M)_j%Fv$Bdm(7)O+I#!#Ir2RoO|C*vST4Eb@juRMP$G1jO!X zEo)Cdm@-@hi3Z&~!?BVb=#WWPLB`+rlTrK616qj9IV1Rz9JB+XRy9-lU0gp^o@^xG zKrk)zAqO4x>t8g^6N*5Md1{w6tUUDhqSO}&iaxkzQaB!^-Y5tv2^&l$KO8sFA#S^SzQz8PjfG#H8Q#m?|bYUBdnj0 z0Rt>l^gYA{T+dLmen8u(3F6h}5!D&D02jo`u(PuV*}Lb3??KxtW(VNJkTZTDqdkCq zOrc+xYA0Q<-l?nRw7vLAUA-Ve5zVRqnK@JlJqt`OkK9AN{Go7dg)g@%+1_{QXct2@z#=z!Y})|OZ^ zGtjZO&51j8L(-{(joBMDWBfM_gb(ov8|e!>?+XNr5O{lPA_K@AQEyW5{+Q+2hb!H$o?bzxgppB1)Z9$6pV zlofn&gPtMpKS8j108$Glm_qbZ7Woj>8Ii#0_i##fK zGjHa7WZ*M$-X{nXzR}$epU4FKM1SC9H+8}I5{#kw>E+ntt+5t2c1zF7D5}iHhyia7 zhp9xwfax<-{1>!i=-Jf+T*VOT*_>c0fbxbw)1;usZ*1MtOSI}HsBEM@FX|9DS1_b> zp~6c{U{vK0a06@T2Nu6TJ_Rs;%vkeXw9i8KKW?HrEz;#Bb{}sOp1pVrh_?hRit|2& zaCyF>3Vnb(dXk@6BM?IeLIGVT7a?}6nkjFmjI~rL%=a(02Q6K$}^?V{=YcD z9ce((Vlh~q)KW?$pZ^HiP;<(yw%y7MCxS9z)XEC*Cn%d2FeHlw(1a<(jd2J3PL`!| zsDrOB5x+Cc@Z2;;TMDe~$BiiEjv!z%PoK8e7g*JM7PDE5x~)Vh0J3iYaK0Qn^=Y;~ zPX%Zq^w&PT=Ht9Dl`a{}Nt_S$27ZBY+3rC=uAuJb9a9Fpz#59b{B@T1X-Cl?S)wMj zuLC&VI@{MCjP_Nuccp#bgg5J3S|UaSYYGCJ;s6Dx-SBJKT6OY8DMvO>rg({ZwZW9( ziB;0xUxPeUB)6|IqRE~>-4*aJkaz=td4U~6VkU{5F2V}!ykV|WVBz77=!Nc76=Q_o z`(PGhKyXRJ<`!4m18VOpaQn5-{OEVh0YW-|FMh~>Hg6ey9^j=}`-#F9gXXUR(nAOp zV-TL-NF3k;v)8Dw+1cUh9Ayc&q6}c$<#%kVF{mp*qyT*lO+*-8QGz*z(6c8Qa0}bZ z$EG8;aAqHN$p^YFyz<(%SIZZI|MJ(`pOF$`uA(n}vxgbcYwD5hyaPkJ7_W^R;1E3?fcbJe``b_ca3NUzq zURn`BE8z?}086n0&HEp+ zAKRM4?fgXUOa#Q|7=41;yw5x!RAF<766^#)q748!FlSNNaCCqOG%H$$)#%8k z9dn=0q}2R1NViUnE>Y|u+5LmL8wsU-0JE8GDZQeEET0g`l^jQ08@FM7CIk^ zs*%DhSs!FOp|&WJV@x%6E0XLJ%8+c+2~*Y+pn#<5{?Sw|!~O{1hKq;;ykG`^N1Ugg zHG5KO>bWk)SORC+pq@ab#~q`Ryx%qG$X<|pNyOfnq(!o>P>V*kp23*>fRo=GN%@f= zxg6RlzmsoM`QQpFv4hkL5@5mW$wITajAlwebcwEm@VBh826T|>Y!0{Ltq=7X@GI%# zKgN)DHq83_KBxc`Ads_Z5gYXwnUo_)Th^Oq!{axxz^a&3LgX8`{IkVZo?LWEP?!&J zT2aBhNf>!Lh$@t`I4lU7X1xA$rI|c89IlyB5^z6=x;4%NoffynQaDiSV+D8X6OiUQ zeF>hdFFXP%w|^Z6*u(672RHm(*xo`UCrlhYnWg%NSU#X>%BOL0sevDDoIDvv=B(Kk zm86VGWS2f{NN))K;K%$s>$k3akpmGE4C+`xD?7_begs+uUBDh$`(2n%7@Yy;pR^a7k2|w%p&7 zEb_cG+6j_hKz?~05CZA^$Ad+Xz5f~zv3?Fy20VcF&Xee+peL^iApRh&t~)Ipy?}24 zwwwYtDb(+W9TuudBA~DT;e{~sYZA-0-%;HjhHiR_D8#4Dej$Hz{QAA;BVo+MKS-?m z6KZ>Ad}XO0)dzhfi8r{PaAjNy>IyU@r_F6*qT0*z*&5rmKD!4^J#Gf@z3D`Wd_BT? z;W}yT$Fn7Nc-*kX<%ie*0*#NbaKO;dyS40d+9E465ehGV`D$IXiKH z3(Npq1Rv^;b}wEJW3QDOn*-4zD^Td=%^5c6>@IH%xci8`tm z!N{O()^BZfj0ofu;#GW+1iXN6Qy0w(A1C<-;4Qp^eUtLu;QzWNUg1sX0Jw`Fc;ElI!f8F;g;`oXOlCE@%+B<%kc%R9{fq$uk)4sdvPFB0Bd{0Z^j zMf3wM7<}_Y=nI_bzf$!mbua`|DK4H(G{XHI)&I?n??qX^+J_OfvFa8=u=WG_AB=lR z{$Sr$l2(fK3}u*XVt(gc2|AWoO+WBDH5QuK3E-y z#XIpBZU6ecsIUj*NrQd7=X66>Vg%#`JH;1P2erDEO<1m8F?hVULvAQbSbe7HZ)|*d zmLePt48$$iuM$0pC_~(gaZ1A9hrNMsdnpt_A!lgIf*nnn_>Xk z2`2#|^%qv<7Z&M`+*IO7LI2C7W7;y;_-d$LLZ)-$*)(?i=x@A$LwN$P-;{XEE>+Lv zcNTAm^vAXIl1f%BqM0)ALYzMu>K2*v$FZJv82m8!Ng4$Hl!VWhWfi-}0T))B5tKNq zt+U4hGwcn)NUHfH8&XF+&K-Y){G{$iXSBCjM9Z_cMjv#v~M}HT-#QnHB-cv<6Yl2o+gV^e8{BW zW*1v|8Iz(i!2w9G=n+?xjLb~-j-o1CKqR=>{RHQtta*;Zi`lrqdTTHvYkVBSI_nOM z%st4Kv*4HLr=nidU5MsEz8}vmXP1T!AZrv4scxF}S{EqAP4)@s80*wN!PH9K1PR^c zKPNJZcWueQpsJyw3Du%!QvA^6QgmO9aVtH3T(D2fwblXWXa@#}B7~jXsDV#B^Ep5(5J{?)Z5XO< zU1`ujwM3nthDS2k&g=mFXhNPo6dalMI97{41W{zqv)9>R&U{>;qArw1Qt~aqt2d*K z<`0h8?5zi~N<;;@f0uw^yNy``ke~CbwjeA&Px-GDC0(ohFYv?PDnu!nISLDx^O;>S zP!m7rb3ttDxAeFuP`jvCp(lQ-R(zcxE=G3>^LxS{P$h>;ZjYSq%G;?=x9)i0u7W3r zA%HvgJ8#VrOODpStVP}8{3bNNq&)w50W(ekF&gv<3v^zsWl+(4DyPmjm+;KrT>eAJ z-Rr!R1Rwne)y{Ogv^rG$(I$hlYD{!PTZNODPsw3%nB!4a@d^s>ghZJeMY1KY=rqZ= zS~0aMJ0!!MEeu8~o8xG0Ru%-F634H#*_XCy(Twa8oVOVi{L#(Mn`17eQGzEe%l3~= zFi$Omb3V-SXs3b_7&Cwq^m5sFgToUIr2xB1FM3k1iTPq1o)kP~Y0i+2GU-7i5x%SS z8V@P2-A(>PFpV@`s|Pa(hP^$}dE{noJ`b}vCeqdNQR-aB_T;xltgX>*6FWnLtA-qg z2DXHg=ekF6I3Z_w#kw8pRm#;EOK+YG0gZ62oh<{4&J2-9&hb?utVszgZ*7&@RVW8$ z<}r6f=(Ji|Z?APyUcs&11a-ig`ED`?>kMdL%c8Tj(z&$wec8>?1(rL|%=PVAZVK;x zJXe$3BNK6o+_I8^)x}(FyVrP*oB^}z+1@O#9Zh_9d~jy3VTl*rLkN;uHkgqjV6CRb zxD`b9w}*GUWbtaC4|{-%4zCAl5n*(yxB&%yHvo@a6=mbxSA+(?wECnX^3|H z3}@%qf`Av#gSjxnQ&t6iq#HYyo(E@u#hceDxLF`Nzh&R_OMVu(uZ^qI!5{+|0Z z3B9xK_t4S^mP6}OR~K#9j6ct}<^C6uxKmTG?r(&XmQxhe>v`H^D0`3rW#3V;q|q3E z@he2IDi2+jX_8$btb->zkXJ=7%K5q#Idp$j>OdFd{v~PpRj-^oP5Qfi0 z1nXh$!jsK%4|`nx2)xR}kI$DP^x=@qGsr`u8PQDXq^5cQ`HUjeYc*}N(5IZn9+Xz? z5Uzp>E{PevRfpTbO#EX?q#e=o%y9Z)z4XyIY;G+;|B~%RmS=egQc+(Mvd&Zx^>Ns zP>kLwUgG;B%;ev&RuD|wVH*>6$=_fa^c0u2)G;zgN04>b{#G#X1@WTFojL_4{!kLl z7a^iWhi3q3+ZYs~n2k~kEzfblL%gr}>#~u|2*Q<~S86OPo{X&~xu7TBL9QpJ!hQ$A z83@ll6{;Uf3H2WPC~a+gs5Y=JZ%CS+0s#P#co53tKP}S8w{bAils8;5vrwO5yB9pZ zr0NABC%XcxY1;esIZ&sA4io)peODeKFc*ntv0)d*sNIx;Qz@T zzzKTQvQQDCCXoLOAXa}r6#?<&Oy2VQxhtVzA|D44X{owjgyz3Nm%)8;2x@2suJQ@{%gM4Fe`ADnkIYV>V`?U@@U1=ZaA}szJpE zX%f#Z35*|b;>EaVI8Y7|5XCD$HNs`Lt>r6--rZQhf7&O&%W_I4wk@>#XUE2*%ueBn zTgQ_CrGqGU0QMa`ZEEECtAsJ&*0N55Z}svFf%1lhaQxc3njNQ3b14fpTGo$xQ;%nk z@n;vSG+j0@0uN7`fYwIqDcFxCT-T={8fd7he_t5iVB^fg9m?))Do_krVsIwlU|t-d z=~6sv8fUi_=5K}$a*0A4JK|+jfI0f$)iHQ^mn9=U+`T^xL9r}qSpZr~Q+715gY?vO z0{zh*zzw$#F`hTb9a%7B&!+^G3|fRBYpiW#`3NWihj`+NbZ~nFlCs)13~f)#zcmC! ztIEl2Z(bgk>gUOJWX4`?rbnGcX|(q57^_5Z0t|KGis;^NY*B*db0x|(;oTP7pj~F5 zZ5x$SMOZlbG|&C>Q1?$TdqlS#(uT)rx)ryme{d0Lasf+?t^FM7U=qqq#8KN-3~BS= zW7FQ*$ECaLN9Km-;{jKt)y4a6*EKjYFx&366fm)2=C)KBKt1$dBi=`qi}-#UWOZq1 zHt5CA>Wu0q!HcwQ&TrqkmAukgj$s+EXY@PRq8ZITCw-0GHXWO03v62|!Od}@P#%~i z7hioOqn>|o zMg|WrG2OPlKE&|fh;6yPc8lP;&nbCxG0dF9XI2T5i>D`5{n47=&JU*qsY!asK~Vy0 zTQ4AI9eTFaL1pr>Bcg!~VnQCYR-+9~$vzFUUI}8e)SFs?j=Ub*!rrJsihBhQzM%rI zyV!|xatX55M83#ezYwINUzhF5?5(Y8$c(u>+#6gHSp#5LkNv5NUcc~dCK)HwC-CKy zT?NRb3p#jJ3uFqz?u#^}a9j>(opVW*wv4d^I>7Zf4tg1}G3pti{JAeMkDnTkQh}r! zg|-*ei?%AlK^mLS55g2deOSF0I2?BxLvWqFcFyIn<{mCs&}-9}v4#yAf}7 zv$++9^(OlF^4;55;;g&=kf4Es$nSWG1Oh)XgGF- zsWWHxFTv=A{dRr?*n_Mg7-&w(Q?e?pz{KKbr>>6BDh-3Tpi}?Vk)|#-&nMFcz#x$G zD3mym#^R;lqa1sNx?j&}$Ar&-w;VLZfg9vCFw@)5{;X)DMqYPU=i?EM5vT&#V+Ju& z?+|<22X359r-RB*^1h3*Ex3|8=Qx#LTGf~aQ9F8^dIh9H{JU04yoxubs6!s^50_Af znH|wJ%M3iDFU~lIFkqP#`t%QlmtZwTxYZOiMX=E|A37wlUV zK!}o7ZdMR1#6;MK>r*ez;*?~-TpnOUNCvAwr~qMRsg?lGwKkImIIR61sYNvPj15|X z26ZUB$Vj%5Ie zb2Wq~Z@UZw0Tg?aYoqboUw=Xu0%V7Ri%!Y5rIuGsDi}OE7*k=-&6u$Yg_p2EyQJZK zh2^0VnFF&|Lf{+(9qpojx#w}QSZ>qz2{-B*d-$y7qT7>e3F7M7U3lA2O^`o20u}w6 z2n1{^0~M1KoNmpz;9)Y#El{&JAEai%0C6oWpt2c*=6mR__U44=e9sLCzR5q!AsHNN zo07}4tev!JHYn zcwD_sYV}W_)|_>gS8FWx$S)|7yk8=HG|VFw69fxW)iRm{!1-%^1~?)9*pzq1ZlpUM z+hPDwbIvBCRr%{Y>zp6AQPRhooKtFML;}5yyy{k6_+Jfiv*jO& zt1yZ)n<{%^JDPj{7~Ub(5%++c_1Vf^y3f;PYfX-&yXw${YQqG98JC$HU3{fE_eeF- z`~yCBTNN_(o^pJn{hc|5h5IQAF#933qEpmtFYw#zx?NgC@Aubd4!eiaa%1B2_WbcT zOY&nPpcnm1(u)(6!GLg-Kx*=5BwD3a&k|mb0!7%7BRFeQD)Y zM;I`VyX@7hi}angdW?$IP{x7aKNO);nDTNpyU|V+PH#1Pukl+@xQ`7pXH|VqOYuQN z#~6neX*EUfpWPOdpCkNeuK8}oIpcU|hhTqvh0AAtoe#BYX*#3H1s?ACdWKqhn%O8s zzNIKb=g1FIVwGQOsx8N&Td-#}r&f3Fya)@1J#E04V8@N@T;858nRbUsX2TH@>Sxjt z#8@Brux>il?wvf;)e{?!7`O>pmY4TEXwp7OnOW5Uhe$^aXLc!_dYAF=Xv_}oV)cVr zrh0m{ci78(oN@btwtgF2!$XZ8Z`UPW-+OjE zl8V5Eo#R*ggJmxqC4D;@F)C@*pczl+Sh}uUZ77s}3=Nyp`;Xd~I*ioQn$J`C;r^|r z8x9z4D;XqIhUFLUp=OGfJ{u^k{!Dg1@gFP=Q4{Wig4ZPSwxG+zcSy1jwoO@3yVFIy z8KEnZe@*8X0wrt40Z5C~*TeWqdl+pE>#@+2^+1t>BFN)G{lcKOQTIF@;Ur^cZL z5t~aD^WmWF;2%h^jC7&NMsqVk^)y178^w@8cFMK9<3wOK&>BdHrBqV|WcS+8sCNOp z^_)@BHQm`Dq&5yfDJ(jo%vBTOE(TX|-pnR#@`d?3K=GS$4*R-e|k@an=Y1!gEPtK2ng3=rE@F zIeIFRQ(qVY1uLO&Z)m$;N_xx}rtOpP-eHulXFVqBvS&br&4|13R~c4s)WDvc=*wi% zKw>^Jgsh~fhR`9%cv9FpWehcup60yYBQE$wP-x657wbg)8+wM%^wT|Cupj-lP{xgt zw!4IaIFq1WZ8|UKOhw&h19p9S3KLNmir{jshk_xvp>ku{TBan$0IlNHULoRcw$i~p z8gs~QxX(jZ&`;ev2txzV4BXwp0P%--4$z)w{`}YGYH>H71+8??aQn{4Uj#q=&pr<` zGScrv_~QJEoTw`piqHjj=T6c&$|Ivg!b5X*Hmo~;MH>zRs!$EM} z9Mq%#O@ci1xS5O2f~S8wGKCH23Hpbicr?jRvG+q)@cQSw)K99 z;!DyXF@%@XJBNmK3{Bq#I_`?b$}x`QXGY|de={7ur$=^uV}^8Xrw&is#H?IdxK|vK zuwMqy1`U5<2U_*7{j67}AdtntD-=q9y}}h|NEfubQilxnR9=ZorO~fKI(^X}pROGD zB(tNSrtA)a`RfTG%$D3>Ad76~&|R%BPEEj|9=xLpMZtjt;U_+C+Fs;m4)schmR?9~ zT*OS|avZ*P5IDxmA9YkY%M${SM1tK^3}H4Z(_IG{PI;N9ieAQFT)cLMmWSJtRmQo} zt)auo(yS)xRbLZNpZHKKD-`%xt*cNz>Vh$SDp(UjK5zg12sV}NVeC}A5 z?>N-zE?9Mx{T2#|Y9FbeVCcI@>=x`}jD*AQZ}gyZxCRSR1XG~lO74i=-#hJ?vp_&` z$RfxXwkN77nUp@>9=QbP2hO_xkNOOZ2s1X( zs+A8-vjmP2W$R???VsZ|XvHw?n;77Z|FcVfIULp7vZkOuTQo)*K7&gh)NcR4k1-wo zNk8;NW4Nf12?Ql1awl4K-o%(gun&xLKiHXSBC@?W|LG@K^Ay?rsEsF2e{dO5GV7xU za3fu^^R6gg2pc7ZU6gT~UW-Ng21m8$F3|Cx=jCDLujlB11mWQ&h29CJRzB>o%8NEbDc_M#S}L#? z=)Y87Q+X56qhDf}H+0KQV*{h<2b*z~)DG@yBFk%|Za?prTyJ9vLPhTO{DU+)_|xZz zcLM(>@kZ}e;#@z}WoRLWKZBLz<{|GJ{zrkGF?bb}!9aheyh+rrUv0kvj2+ymc+LN! z_wD@N@$DCd+mk>>Yy^pf)Sf)~i>}WT=ukmI8Se!<&H@?PHA*sCuzV5Qt5ojEAzp;s zFaNN`&akC#wsc!N8Mp-22TD$07Es_}H-S79#DH`;JRuq9XigGte-L0-WDtVCW{>8d z9ST;kZSpS#4dN9ZqKlNsMH-q4>`yGZ&u#Kf8(Yth8&UIiyPmrV6x#*y_QdVvj9t=R z!`qp{Xy`$krNx82s!b%xAHOtH1uw`dbI2<5!0C(;&MJ#-ZEz9fHO9|NWx?pR6oJGzG)xN8g%Bf; zfUr3Z8Cs}O<2}ob_`KTc#%wk_!+zOmplSJ(_=q%Xyp)YI>o+4=RF|w>=EYw*!<)ey zOw}!#n#z6%GYX4ItDEZsOWO<^_T?eELt6H?ByjsP>C#I*Z8xs{{Oj&(5)zoWO z8kJv!?WmLNUARxrVHh6IkG!bay90WaHHoLp6C1I|7w;?jU~Uh^irUK9R*MlxhO6grJPJ4N=7qJHuJVC7lb~wjnTez&l;?6^+*a$ zy;DYoBTfrSTj%Uz4Y^F0hXziYS+_}v*G|gK0I~(e8KKaC_y!ay2HwVwl~f$U&ij@i zQbrwInneR|8|1OUARMYr3`n1)eu>cPOFerYmXdys7xjKNj}F;uff9F&b4se}Nlq+J z5O}F!B&)`@9Qm-{rpKq)Lm=`^H7F-@$N|4-ju()bPY+^>ecVCNsM_TJiF0-Mbx71olap&M!pt!EN8A;m8_tsn83cOv^%76kQXfxc)nY_x$|G4)p}iQ&hRL!WZcmV)ARTu z-ax_h|H1q#HkhOT`yi#&u=o5gF%jDVra^uDR3e4XaUW{_7nRLsgIg*JviDpX^(cae z`l2mI8tvp)FKXZ7))NhL!K+?9J8?0h!7V~XvJ{l)BrQ?;X)Q#tx-!z>TX~MSAcTpM zdTVCHYLAodlj7**ammSmX4xYXS$);#@pJ zeAJVm!Z5z=gU}rbf6FI-@{d;QX@cW-R{M_1yekY12leo$yf3Al;+RCAdL`Fp*EDGTO#`Zqk3c($5hh^r72x8S5 zZDjdvDqIz@B_woAL2Ytr6V`QkfoQvUT04ad6k`}o69^%+Iuy^|b-8qU5zAuTZ>M%q zzT-@stx$X-VuHc%K4}|hbx`xF?Ron-nA3VoLn>Pjyh^NfrJ?d&X`*G4^;@^6G}pg7 zuf;{Vv65P(XIJCRLk6iL+Mn?)a%{!v?{id!#fG88_zC@QT&qYb!SKFcM&yaY=Ku5M ziLHb;Z6_zj3XbQ(tAfDv3;j||J>Y{^u2ROcL>@SjHSO77Z4N3*Q?V>$VvR= zZgNgW>g$J*;s=j(|B{CC0cZk=Oxe&C+{7e3uLbK?vI$Ex^`)ZZHty~yX4I7SW@%2G ziTy(cE^DW{fyg=NJ7=#|il2ezKEA(?(#G&V)|-rKqN$)ySznjkBX*gN6d)>1z8RCO~q_J8CT zy+5wm#BsO?$y^>7jD*udo*B34WFrUmo@4e>$8Mi%Lno!lAoGzxc!oer*H=_CfuFwNiMX{5gIP?w}9k?AB{63NJ>)U7>x&-cW4{V{r!A zZcQ#}w_%}EkK+4>rmghMU&}2Ry4TY{vzj#0Sd4f?BZfh;mw&*|aEknZI7?@eE+bmx z+HTXRmm5|j+az)=O0ZX|-tMrhgcP_=jo3j(W=i96&V_lyQJ_$Xi7x98VBL_l@1yr7 zqzfsNlYnLmx!lVyR)F^1c0gwOXQigO97k0n^1?yir!5Y{r-tDi(N+U}+cGH!H1-B~?&JNzyW6EsZ{|w4Wkic!N#G<$ zux-?OeA7lU~JtB!!-?Q{OPtV51LQY zMt}y{G%PnE->NrH!Ris{KwXwoC^zCB?%Q4MZS&;h1Lba%c|L}2-Bnf+1i&=Ec%00V z6)#8W*;7=J;56!Fte~C~1c*Y4k0jSRlrGdbCT4~b_c|)ekO1KuV3^0t3PLRduJL;? z|40(SgUL49VoxIyMj~_N9!@QFEWj6T59Z_vfvS%fnY%9EG;4O{?|HxT|KaZGLx6~# zO%Jq11v9Jws)KB}S6Q=e2KuGN6E8J_r!w)YU4)`J@fpq$M)7U{t2LlHCzp)LVoKOG zdv7epT1<@9X$mHhm2QFt*))n=hORv6BI#0>uD98#(9<1WH4qACFg#s*WDt8W)docP z=FStjf`8;xxJwc|_~dJCgX#-jXNQ%lOH2StIn^sks$&o<_@~Q1IVk&s)4X$D%&RcK za)2PQTd;2JegNtR$s-!4w2#PDP1;>9Lahy=E@b5v5Nb83caS*MKBYMnPuK8jh9y0o z)3-7BzTfjqD*ZS!d4VP+K-8iC&p=91!-zmhFEBS zY2EZSp<8{I7LXCGkkxwd19Cb2VL4Z!!n*i={lisQOy0CIC$ zm)K#f2DFW#Nuu|zkZs>b*$;K%&Wf6wq;m?87E&qyVBh7s(sEMPFcV2z6nX~II52-; z-}EQgUzTB++i+-&K}{9ZL`T7NQ^BpQG+~^d5)*2ghai!8*K*{pOmLh}A&Su-Y6Se7s?WvQ z={&^W#ivn1I|29-9S|D=p(<{pLT)DXWXoj{qgO5xD9xe2k40Bet6OLjTpKPstWvHv zBMLOQ5}AZ=fg6M#w}-$x3LcC!ECUav`^@Uaa$IUe=oL%Y4%lNUA|E(`Z67bQZIM&i zW%+AU8Tzrp5w?^{e=pw)JFV>p1!?#FkAdBlO-2NaQ{r)OsDknA&6hcPFTID{HTTQZ z80INTFq2v$wV3Vz=4X6%%V2^n#_(kEX4t=z$es`*q@XYf!w9w3k>3KpeJ5Zbo7X|q z0*Ez!&L_Hl02z0(pPC8c?NM(q0~^}GIADd0kH6FFFS(r9XUS0p6-Cbm-{Gb5R5L9v zf`dCsg^rX;Ts9^LQOhAgixV;JeTgSU&O%;KU}HtiS4QIoS}-O5>Ak>pnHixwdMep{Jhj}h5T_q}i1yb9657-e zYPT73Mx93e4xJ+T%3BaBfoNK!8bfUR1{}c*5&2IAg(U%SBstP7^_RY41dfuC?4KLQ zN|i-XzP~l^qNoWuS|LyM$ABDcrD`>D8EyWJ;T482P}cP<+{x3^aUl^7EK1;r zOMV0N5O-TCO-mndVz>?><~lBL1Pml6a%>qKJm3!w#wb-99k4+O)oAdcEy0MGf{rbm z1u{7DYH%Q=hTLEX{8kSof`N$uB8f)mpWr2}SQJPpago6P!P-~FMB>9+@M!^Ebi_$xGnB(i@Q6EySu{z`~UWPyf63ZCQaHjO(jj+oSt9KxlAA-k`La^ zqj$nKsV_9-9+S`#n`85}Bq3Tp4gZ*l?nDn!uNcIMSDNOV^C2NJ@rA;GQZ*($CDo2H zE(|v7V?sm+H=z`UxpNPB+zWSFiw@raDVFR`$_Bh+2EI<4=mqf@pUM7vY5~RBHkXVv zvE0FZK_aJ|;by}ff)tMy?@hZ#5stYZt-45^m108so+B;fsEHB?S~@X^XEI*Cl9E{X zSQtRzzw@zZpD507Jr}WKZbNI_1T?$VF$mh`gl!x&2Z&B!+K{6~+9`X>5jvi{Ck-iS z=Au7X`6wNW^G@8Wx4HIany4g?)O3tt>4d|mq(kSekxg%uKhS6^_scW)AX0)9ND1v1x^7(sx@stHt~AA0qcG(qk(1JUJ>Au>{Df49Z^yC-6WAMz zdi}V4jEwqKzyfI7=o+WipA?oqjG2&*L;Z&oM?{)CVUrNDF<)&;-qF9xGNFfKO7-kP z9k0>EKLi;))6Gk(M~Nv2j0up^cRL{D7?3B*jZQHp#3|vNBgfRZEm0!-8hM1%R?#2S zkOvGLkXsqO<nAe2M6nza5nCbKb77;G_(JHm-S;qM;`R zCTi0yOgFJ?IgwtD&Z}GG0ORPySC*xzcEX}t4`i^|V}0n;*w6_ktIAvELW*xHwL_DM zmppM`FdD)*FR_oJr?Y5wl6W2zA=x9sK36!1!<5Q+nxsSIiUln@bS4}v@D1ar!>M}- zM>PDA#fr1cRW;UfH2%rJi!vUxqVqRrvJI}RKX61jXChenv>nUpyXQFfH)0H?^(nIv zI3%$sAEBv23`T9E{wfZZX%sNrwKVV~D42KM%jc3_?^t25YFT2M96@N{TP(K-gqov3 zmp?3{N!1#qMsy*W`l?tU7HRaR?5T$=KkZ(oOXd|T)r z)WQTvv2m;b<=`n=l;z?8P29jqX1RR`8`ale|Ne+$wOMre3x3lzR`-B{oNIp|!%}>p z6?+$S?@&2jV(2Z#yBTF=0zH9 zRm@mUNw{U|$4WWUKPZeNoZDouGK@hQVUm$ftLIu}B)&~YwQ0?)emD@ni2_E8!6?3T zrL5bUkrrL!T#f21=80q)Ca8HIuRN(W_?%J$hwChbLBE!i84B7YHnY9HQBJHN(vSTv z2b>&@YSQU}k6Yqm5&RkEfi11GX=4))*MdW_wDO8{Uez=(Clvw(_N8sQgjp9*v+g!s zXZH{E)+cD*9V1D?jH6JKf0!~FA(*b={T30knJ!TXCR%5-@c~A>u&98Nj}hSl=~}Vd z7XiTX(`+IHMftFRX`*kD|EoUUPs%b;_MJ*IJ!kC7*Ye6VJ#lhuuIOOtef(9E7Av|g zFlYpiS8Jl7Kg&wVMm*B)$tps}r0$TbS*$M(I%8_cplOJv^+JvWxv@%?1q9}MV?IwL@@azcCFXlQk(^&Ox>IT)}lnP z*J962Xt`OXnmq=7{E%oM)YY5$a1Amsrz(Sa!UY3Rg940MMB!G*@4`rtxX`|+CAJ5m z_(u-Le7oy=yjI5BUL@Li?B zz+*OWpc_m2BK{{eW`e=7-F2O;C#>Upv?`09KIX+uRB?rsu}=}0&jnl$i)4t-T8)xoB> zu4d52fgQzzr@xR5%BWrqH0cZ;mXM5-EW)4kfQ65m#1s3u4m+VWk?M_rR;K~DMJLx* z_Xcqc`gSPRfs~b7%$G=z3I*s?2^2S*yof4Ihg#bAb|>Xz(ZH`hQ5ga<5Kes;2`;~R z91bnZnoYltS&^|*tw<7n|KX+Kg-`zqTJGqhy+V;6O|(hvzKi%(EM~agMj@>o!(mkL z@M%&s(Ux2KBp^}0u`eMQ3mgCd$i~=Y!@_BP)!2b~A~ZC};0*JI!D|e=m5fCHVzzW@u7jf}=6ozNV;WO-co9sq;kb?tA3bUxqj%&qDm)ae zl$xw^d^1t2`38#iKpRyzIb-U6e37A^#wNa6M{AYFu?T7+Jdr>*@rtR)Tu8Tcm z1$!yxxt$ZE9x-~jL@((t9l=s_UuAQGWvaFBAx5d%Fz5!l(K@R^8VZxvMDr)Zvvk5uvu3|plwe?v1i(jiVjWewxDT}Z!`fQgbyqePM=^UugF7QHZ z;RrP=sj|RJis;jgFyUuER{F}h)S!r`C=smE)R}&cy0>>)UIZe+%O1=JdqdGZGMi-G zfRs~2c_CpTGzrv$1mO;&k&R<;e*y0Z(j=U}YcNKy9Fwqpe!Hc-{6(kTcOgspp^AnO zS|-7Q`=Xs=q{&B{Q8089rhczAMztf8SxvyGSmfy+@|TNR7O|wUkdT^rY&b1j;k#&nR*`vaxB3K;?5s|}q;JMI z7b$djtT*%$LkBc14VhW3-~!%1NwopvES*z5VYLf1>S z>dkjw$a>i504TQKvEWsK;+88u(0D~cKv2J8qoQ-d%zIGcOC;=IY|v#RT<*lufNH8v zy@1yH(tlpIWWp_-YF${j`wD2f8}aj4@%XFrxf>O~(ar8DSCEYr;u~Bra2y!4--TXn z6uQK}?PTuG8U+0IUN-PH1a}8D+=mq&CgQaZ)xrv^%ji0x8gJ1U5#IL5sXl_L%yr@; zmJK1^dxWUi-{_pWwh(fgKwMwzUopyGz;60rJQ?JtFHMDQkekABzjsk*A>{2B8PW4!2@S;N2y)KcZXgo7df%-t@uLC!n*_#1)w6mi=aj zo*23q0K9?{L3Ayr5oVjt{Lgd$?x61~1%yd@{Vkm8We9)`zv!-_jMH8H)3;5=N~QG4 zNvAZUa}R2g^;4)YstOYG;3pRe$P8!thJd$V0nA{SMiQ^kzW(OBss8t>dSd4LYb^B< zoc@VQ+o7Ybmr*Ze0_ZsH0KlvQi8)|ABV&)bi#KM6(aNHVF|y(}V?C>5N#AM$yHESI`i8f~>VW5_saLY*?TS0$W4LEEl4CR0(vD;aCLaZ zM9Euxdlw34VJT*c|M5OkU_rMNJ-L+2@5N86?_0n3cz1I**OHxRD)e(LDK~5Tm!@Qv z-_pa)#1$E}r)AiO+9y9ZwK6;TwigVVlJM~N(XgW4MuWdv&0!^p=yF}HA~a(Vms=R> zhDFGaZA=EWd}90P-Z1{M-?NcKzuRW@1ISy=Kxpp5e@Iu8fFYcR^n}a38L-o1%s3BjFumuM~Q)f8(PvLN0sfr~$QYdRS*)(frX?@E3 zjh7(`G6_P7LP8wy&c6)ycnxs#2*J%1T*ne7?xv&%Tf#cmjmj%yG2X0p=NosgaI`Bn z_?qm`F_PB<%JLVj90Aw_fPLG!*(D5d0r@>J3c6M0)eM?Tr|esa~>q|kvXN0eZds#FO;z( z2ZEGZWRO>OqO~fIxupYfq`zY>KJ=|x&?P)g$hm9B~CoI2H7slTpEp2ECR40oEi%}V=7m_cU)|g4>u9Cq}78-tO1Rg}!=b^3s$~9aG zGOH*^hatJDBw4{zi-R39|DJv7^!=Av^R-ILFbCg%$)bN}!ohmdsSB#QhKrf8I4JU6 zW(42*lw_otNnZz~#BvEKoefhmm^8hGaM-r^%s8l) z4Baqkef5906IoLcutlw4sRmBy$F+}1m@II-W}8~EYqeR&3{KJg`spCxKTh=?7p=~g9zok3S!7@a*U-T zB#4n&fF{Ju3)JJk@}x|sam><~L@mP(5aem$h_R!G&b+T%`E)DG&tfeoPu(VzajB_N zd2vh_9An5p&BeN2Aoyogf$)_NNu% zj2K@YPv8?D}c3kTkXg}F!DhX08165*l%yuPB(-9*yAEtWF1@z;Q zfx|f%>8|ihi;>xc;qknU3}--n=Ct@>xm2T(#BU?9Zv3?Jou%+}kOEc1#HYJl9o8qE zJju{h!Chd~P#1R{0u|MQv8TU&hvA~*VbV!j#=r>R#1>!E6S+w+KYpt0QoGg?T*)RV zfNOBslddN7a`ck1A{21;As4i?rjN_BXm;*b!mbh5lv{b|(f-X4Fi}1zSmgffFUb!; zeZp37Ut~AVe>M#l^w6@Bq64cp{|@zQ+-7Qg^{9tz!sl_PutSz3$3E9oSSOn`p23Iu zx{Lf4A6%a7?I`qveSu~_|8tx4#QoZVDpIqrQil6=K+D5G&U>5p)HKo46sF@sm`gYd zg}=H~uo@S^fVKEoY+YQMTbF>S4910y_^ZEegJE@!xFXa~QjC-~4&;xnjdutru8&pxM_xWUZ49Jxh^hD+o<@O zY)k(siU2JT~Sm+o>h=J@Apo7kb3euY`l&! z%QN$ofa-9NsOR1;utnY$Qv^HVT2jG{*Fw@a^Ptb{l5S?2MKCJ_Rps*WTn=*RbFkz- zoHS-MS@By6mRi;6VRnL{F0R-&``@Rsl3*b)wFdFm_PnZnEoy|8_XQdd-#2-SCDec% zNclG`L=}^%dK8YoBurF*HwXR~U{OV2UP}?Bc{^8NIz@ASni%NI>PmWOu|m4KxW9Qj zg6?n16pC6_Be^SXHf^fKSL#_B*7 z#B;qP(A-n*V+z8=3f#aXJ%AM#XWU2HCc2F`1TtC>SYJ`e5BnyfZ;Ab;=e~89nm@zt zX);ZWMPQtT>KSQzjSI0!7oA%VNsPYBb|8=C%rz{4ZVXKLykjge{}Fx8p1A>f9vqQe zj8qTVyyGl6Pbg(Q{Rw1zciZdZOzIQvLi^_7$iVFeh$!?YkyAPH-RbX&|8n;VpK2xN z*)hZz7}u0YhdUE&06QYw?yfZ`=YHj0hxoT7bx`nWpw?P0T_Q%3uFySSkT*JY4ABAg z_W`dW;l`%+?tGAZGrpC5X`sMQ*))4VxB>W?ozxtK??0HWO`P8eyO*QE(c}+bXIUem zx@QKoMy=NkhBO+o19?VO+ScUqSoR}H4*nJ-z$CfObGhyn1iD>AkA9*)6LaYi5>XLB z%jpQL*W4Jo=Ttsr=pi8@%#5{>8X=u#J3D-Zoyzl4;_hr%-UuuHquQgAS{46=?XQ_q zOh5zUURa1#2ka;dZ0V;q)G$Fj1huT8`;hI2CmBPk4h=>V0ihO;V-5^~8Vb-s?vH`G zF3_BG5)(GQ!Awt67`7ZRtmWdptLs0x9lAh0A1O_-exDz~<(> zX7~e!hMrUN&rP7ya%#P}e}?W5>3(m)AiJ)V0EUd#E?KWqHCgLi1r)mI*hT5{fNqgOMUT^ij^V#39)<(Gfh^t#;D90KDo}2wy>?UoB90Z%Hp&KUf2;u?%zuJaPBn4ITW7kZ)@6L2xV#s{$Xj-+`Cxcs~);Vw!K5*#9xuMz0O!x-(YL% z0Ja-nR(2(?qMZDR^<`_F@I5qbT_KUmGNM>WQ@u_ofKg!E8M&?S21wMO%JpsJ%5$!Jtz>aXNkHH^QMK`03# zyn^HB_f==mp*nZxy_iwta*q?3>wG)EDt?g~Bh9&Bsc+@i7A7~&ByeqB75=6vIwWu! zRSp8WX8Sb(T;n=25F87a_)B7$UgcgA%B{MM>`ePAk|f6<5EDNXObR`!uQTHm^%4s+&b~7i+}28Ocj(KuD?+!SDssZlcggmy_fHs& zRJfyWQjl{%4l(r9tNi0k_o;-A&=`OFe9IhNLICw zN{`LBa&|ri5&R76{?GTIuZsuf+l~GXeJveH&3xDPk0AE30aG}pC(Ha$&lcapTevC5uZ&+4jpp{ zpA$CTC`nQ_5P&t403kdLw9`Q*HY~U|AQOn6v3o&D?TY>KP}R^l)|ia%IT==bZeUNd zo7rU}SfeLKivOkOMEllD=fIOyS`i~BR?71Lpe=(6Jg(Ijc1duPvj6xoyO~*yyzSU-9EScKsE-v@2Bh_-N>LJT0!16SQ*RBN--D9NQVbS#1 zv88&B3*dm$Dif37y{eQ28(mPm-HcD%`w^#{`OxjR>6 zzQ7HvL3jbZT8F5jj!c=hAv`&<2>YT@Z}`%e3c6}*oolD$Dx6AO*}v~r0yrPnY^0@C zVg=T8Yd%v1&kyVJxW;*Gw|Nn&?#`I%QkeN+z({WzzHTBViTminXVTE`pP(bsBT1Ol zQANEFsW8wxro<7fjSl=#HJ(i)_!4(w2cJ#+JRy$Q$s;KcP|$r}25-X*$k+G=ZjBo} zhZL?E6@5r&sR!P}DB~-|OZ-)cz~+u|C^zbLbSO7wr>4cVFRvO~;1nhCLwZyE8+z}? zKiMj&8*r>A>njm>e?DC*{=pT>|Eo~Mx(29~#82s1ek?r1$OQM6t^JQo{wGg*bBreS zy|)8Q0sfn~;TJdUqMQeo{qd+t{mX}cCdoyjCME@4k&sFo>(NQ-id$}xsc?a?qDhTW z9M4|rX-MiZuv3X>jXEJ2gkw8Gz!Mwq?f_fv)K}!A4$n1Dt457`)QQEb(g%X61c9Y%Z^w!w z`9iB^t;vRsuYC+xU%KFWqH)ie*HT(@`l>D;+G?pqtzGvjbG7GTE2X#O3cm~V7P9{N zE{I!%!mDROD)H6U^uGnd)>VeY7iN15-s$X5s74>+((%7FR~ueXdhqCr(V3s~ujTiF z_uW8Ki^t8%R3md)##xufN6HaeDMak%&=@fO#q}bnut&M=5A4J}snI5sjbAYHvq59w zd4bVuU?W_*yJUqlt0Uv29v3VuIQQ*Ujd4;n**1=5-lgg+koxSdxuXzaONWmFoE~4# zu2_$JjXH6!Hq}SXU@_5;J3+(yM~0&(ptnGT-ZTIkuY8qc0!8Uir7rfre{1={x&5I>tqTFpQk985;d1R4z2W!{2>96ve^Wi_E_fUKL7Y?l4K*Gx!&;S zzZc7 zl=y#PGVp=5Cb~f`6+}E+_{Xo@kK%kMV&tXp&Ckk@h9lAo?{YQ2voo@pG_ z49}l4)MRqNAd_4X(!uAV%le=thiVIwxcmMc;Zh{le>|p$X(KK6IL+)F)?LU+FkYBP zar$hy8xgd|S-8@9b`@ce)#!BD=-zQ8jCVQVvl*?g*P!lXthaU{oaw=R<-JE|&#+q$ z`u=ZguWmBo*<8>FOwH*n{&K}0e*mNCH~sjYFY>6L`NBVp|+e2M?AfDUC#Tccw|{_!bJi|Og=2{q*TFI7{RP#{=} z16837!o$44G8!oB3{+vNUHkHnKf`D;oNfZlbR}6>= zW}sK3I2s9h6}IFp%qr%{b4u(G3kvA49}&T3bokX;_phkoUr}WgjFR(^5ytO<=i1l< zq4^6=cbbEP`RCAk(!Auuw!~|qpiq|m1E6)}p$w@{^R-O&y?Y=lu9pW23$a{-)jIlK z8L$B{>K>XlHWro|e1FhxRHaD8+R`7EP9rBEJ{3Zy$cQ#4T$Bbo6iP{q#T)3v=0{FR z%T$!ks4hgV_THc`yt3?bS3mbak?VALTw0i6%54})bd<{mhahT z>I8x(Cf)j*N+EyDV$tI%>DexLgDA;=`2@BPk#3rmJoW3ifOlp@qEXHLE9R7zXYw0R zN162+;delc)6k%EpJ$>$H>&r9xAi?aAsh&uKCen1I{d;`O@=~1wr_)dTY>zo4LbZPgO4{;ci$mNRG3wM8 z?6t83HWQ$(;2rqwm!R6AN&ov#Y3zf+V7wd6j&7J$>She(CSr4P-!J4^P+e3rMy5GH zI$G2?u~L8XPWWvdnd>Jqe6+2wu`;%m-BBE&$D@+N6%-fqf%_;edu1mj0_8v^*4dhW zY5#wQvsw_WApUWsLaGVeC^qbpx8c;!FXcW#7WIgU@DIL&kb3YIQO}Q zjDtt@aw4>hX4f*w`5T#fiDvH9XYwJf`K{<-99qRW4|H+ZAv#_JmWK4WWOJWbo5F(Lf2pl4c6hsjn5A88tXw{2@Fu zqT(KYK+6udgG>dBy9y2t%VF@5O>VnLCPEs^b>$WND)MNSF1L5#a-y-L{5QK}q62dd z5kIVqyGvBEivzsbnj2zt+cA#Xorqz;H_9nD$!|nBo|MbU=rOQ~KzfCQ`u^Gy33nlzwY>ZRj%nkE#f7odY}O*Qi6@oqS1Q$fifD^rm-)Rr z*z%RHJCrz<@7P78Zc>fUsG&>Wmdk>DA%+LuM(je`o>-xt1%7YPCg)=ak`lr;yrh(+ z_iyNtITYFiK#5W&C@~`dz$d8v-72K-{T|A>ceIrB52#y0*~tzol(i=$O(o#(l! zbjpgJzxT!*^9Arf|Do1kA~U}}sfPMzkYuV;Y3WC!(Sgk$Sq%q6G$9(%DXJ^yXy&tI zZDm38ACd!~8JT57kdr(E&zWX$H$OH|oNTxV7f?$FS^k zOf%*G3?rN6(7!XK#j>Q1b$99k%qV~d^Boiwk`=n@b4Ka`53CjpP3{REAmH`R@Wl)>4-#X<+?E0OXTJ$-lNMAN zX=m^#G2ptI$I@5zLPMnG2qY59jO{=K;YRZI{_u`DiAu7R2LtxNOl(Xf>`%5MZq@A3 zpN78FtsG}bF2>|dDri-ldH0k}3w#26L$HT8)dT!w8Fct0gAZXDv-m!Ghjgi5Ik{ zhAdQ*s$_N6z{U1|$iTKlb8n0jz1V99(LDA}NfPZ@gqRGUX91{W7FOA@YgxuU9?k}E z7_M$%64R-d1W8)Q$4U||vI$kH=lvGi(?%m{4VU8el9fv!1g0GS0TRa+BZeIc(kak@ z|Lpjx1TGKD+kJ0@?_^(6)+PTbF;LqOfR$5SGgkxdtQI8F!qzrOe8XN8Vb7(s~T6QFOgbUBSH;QbJZW=@@JzK?{R@nFWA_HFoQ?&E;?nNX`FDiqZ%WjMbyO#Z>jD{oIW zO9@{^bF?*tPX?n32RpOqV&vRjK+#EMPiA~p)3|xCefBEca2Wu z__e`2%LkS?!ZRV9*$YO52doC9mD(4U0X3`hxTLlhNT{)Q|l)xKv{o<<9%T}%)|%X zP|>+9#5@49C7Dy;)e`cg%M<#ev=j;?6_X=|$Zry|o|724#OA!mLL_zU38Q(x?srf( zo9U;xd*+s3kB>pm6clc`|9+9?*kZ&Lv6OyHl$m;1BwkUY+w}6PkXtl>oU1MGS*+NY zY(`Z_q~(Am_pxsN`KKIshnI^kivw3Tui-WhT(bY*oifGlR357?i88n1>@$AeJ^$E@ z>KV0^l@ATjZZ#b->zP$$Btnt>s|lI;d}Z>XJ!$%=D122C0Q=w~YaUvzuUn;1$6p%z z?+~@)RwVeK90*%mE|^VWJ)~IvC`bC{b?7NS;YT8cOA{;SsLJb|naR7Aww+1&wW$+# zDk0}UimyIG=28wHoMlh0LXM2udgsQ_-Md6(c1eHBDup)0$?-F;ZYYVIWr8Bh!I)|k zs$+33lh$B!#cLsRRQ{&5>9xYx@Ej<8jbz@+3WEJ~nUS-_`Gw7K&*D$MUlP=37D_E^ zoL8RyCIwmWy!!Q17!$7548?q~2IOV=A_W`9r_6KE?W0h-(3L^IkR<#V*v-^N=&sbppjKhpxLh<&CKr z5#2Bxx4f-6n_x;pQ&>uj@W*~{sm;?>GXJ2l{ZFt5gwK7y*6f?^T#(A>T`C?ibS)x% z%YwoK&NspuL$S>14Y9^~OWoX_&;BK*Y^ zz}$ozlK5Zes?oPXt0GjQmy;L?kwuke!N>zoZgTC>E4Yq_%YL?+lFDtaNawsCIjudx z!Wo324zq-be31ZW+>p5|zdWH{?ktv}M$+U8wz1?Z)6{9B{+)vetdAgNIvAo-@93~? z-W_R37N>ZyvX8aPba{Q z+zecrcV?`GGE{}`$=^ea)G=JrVd_ioI7k6fCb(Lsv}wp)Nrlc;4FG@_)gq0qkpr!M z&++&wK1XQ&0grg_=4g_zpmDG4eGV2#Z>Dc4Bpm*R^jFsdhN)PROyf#*l88zI1(HT} z&KFlta`UOaoA_O>zj0H3kDx?~U0GsB2U$v(Tx>2ggh&TkY-(Bo%ZFxUBIC-xIkR04 zcBX7m@&$ECEogBR1J+Vq##-+khbmD)p$NKkj z1AxOg9QoWTS5l-JO&Jd>X}b8TE9pCmzd8RVHBpS3lU|r3-&SR1pUD+9s5GtlA0NL3 z^Xd;wadZcIj*FbEu7}fNRhkSQ6`AYz>M{yvhYsA_wAUemwYzmx5&H&#FxJJkSPBT#OY<>Bjye*Jg^wlmGuWRu zPr5g9U#J{vdf}P20XzWT4%9H;Ab@o5$>;gwDT)U4zhOq6bj?ab6yQ5Bd{IoAPsUGp z*cEWjv($sR`_Yi=5NKA5Ko+O-NWWCV7VLg9${O3#ABPufG?oL7JdeqC-Ktp9+Ef(E zSL&(k|JbE1$d|E(8?Pt6`7pZwmrFSRaD+3ZLLZ$->83sA^#{)3s!kmORY60?Tm9&R zmrSg+te^t2Caas9vIWXr$7W(aWCjOoBNYvFt&P{B+crKI#i1;lF^Frf_K$yOq!j(J z_DNKUZ5d+anH=(3j|LDJ0SbYHzmbsWm#*i8Hye)-2%7f>@ z4BazHJ2`rz;z5hKr0+L7|t z`$9>XR~!sc^BBvezal?olbWl|-b`sqM5d!6yJJsvlID<@1JZqHH^tsH;CUTQbehd& z6))6Z;-wZ5?@$NmD6}e%=A80OA>a|`+B#LxK{U#LKD6b=x3*X|yfl*sXk%9_R8z_- zp!5glb>H*XakoF$EHAPvu4$mzTqt!wOr4gzZzh&mLD}L$-Xqp$? zks8V0fTEE+MW8?!j?&Jw%5Uk$rE*l1UKr5u;fV#*b)ziejXdlJ4vN817i>SEBH_Qc$>) zlGg)2Ji_c0=@^t(N)OusTR=`z+-xd;SO-RkFvfP@4%RFOSyp!TMu4&+^q6=ElUe-7<1zDxivJ?jCJ$ExI`Xj(<2o2)`JWF*Rbtc)r#9zk4?d{zVpH zxivyl7n89v#}E#H4^sGtFSbq%$l;=|`6aa(&D}=b{bAFl>)|qEUR94d*sj6G*#=4Gnim+8xBvKr9W34T9D z*Q=A&O;NBCyjg=tdDh9=uC>2P&rj9Z7poEl_zr zZ?adv@Oa5eT+mM>nlnn^?qqFaPh7@FR0ehbw;P6_@+I^VH6_~5ewHd~W%kGJUsvO1 zF-lK6j8zU*snq8-W>KGBf9D*VV$Sn|qB1_s%1cfucuATP{3Lcwm*hX<)mYT6_r9?) zqfvrJJH#0Rx%xZ7WV3I7R)G5>!hi2-Vv*W&>Kx;4`B;_Ag1{u@|M_@_s#cG^a(B~4 zR56Kw(Dn^@fITy5onL^0Qw-X}xn+FF*#rw`@_0S2ezbfXOV3ut%(&j^!C8ptw}DYb z!F7HNw7qAqAz&y#2H30I-SAs#fLGKScA)DW4SqUvR*`l|AQr;*)!16;5oGU zy61r+cXPDo>2YxuXOJK+6n6S@9itR&mcaR!3JK*_${P<_#y>p7LdY<4fZSiSfqZ|c(;wpvNBvHNlV{9cWKXReX$3;7ITCf@En$Q{ zks{Mo=E=KL$iR&9Eswf{_a^@HocNyf`yh2@1K}kCRg_hJV3Db9Q!C*iD~DQ2FJGAG zVore_n&iEdwNPY!AtZGmJ1^GRDu*=VZ9CsXXjt@K>Y$lnHV>ZI|0Pn8?~^jaq6wDT zEZVj4-9FmtFX3j>4AL;VtF4Gk9Cd8&U6JJC$D!aebcbWifpFjxkYQH3F!yY-!^8qS zq1X~E1T%bRF!m%i${{Lcv^#VAW$sA=k>oc&Wljh_L2lqej*RIT47KoCXa>&%_IOiC zBmi%i?yb;!ifGmA`|MZ4WTR}kaiusIk?H-$j}HEK@$68!s9bD{tcY=?*cwq{z3gMb z+B-?#_=5PLwye?w>5y6x)Z{li1r}#gH z+^XDPpQ7k9?BE6KVq6sxLgBn?82a9wcR2)6FX!R2wg@jvg{Lxm3NPJA;ZKRS@JeTu zb`?*s7zx%Fz^$|xIWyQ$=JB^RR3Vdo*?Le`!)Ss2;VSaYbY+n&AXDskSlndyq^XxT z0yT}WKhpU{W2V9}ny=~W&FeV$zCaG`{n2US51WAh#iOnGuHaR_CVL(k`6Yu0%eaQ+ zpX9=dPOe#O5-fFr^y)py1~QcI{8nNk*Dq>>+Jr8v0|7P(>(h?uG{aS zXj?uAy@bhXiM@OW7CODRLRY+pR@CtM`0XZr>7zY3`SATd;xz%rboCRk$-MQtATVP)~u=o)52Mz9R9Hi zZJ8_|+~j;6J5ME|?!jr#SYDT{4(?ldCoB^m>^L| z@4wnbS`mWl@9t{)MDKr|NlhF;;fsxz{^lCq$99*aqE~wNlKg~*Uyw?X0?1_k^zrt` zrn%H86A3y2n#iqZZ@$wQc@5q9ue4-5viS>um=em%VyqofK1%DU#9uL$*H3pWRh2PR z=I>b?BE57iVzT(1-k4fV8Q%dvF<0=rS3&md~ciGopwhkJa_hX|`6thnni z=|7MA>{;)-rHo@M*r`Bl_y*$<5Pt90F!}r$Zdv#9XQDqmQ!buCn}3zH1$o-# zv!)ri8{+LIEKm>MmP#i*?>^gf9mtgE0{FuhJ91C1^pjqW3+v+As)Fgd2W+d$XlCZ>#2HB2aw-f#;N{<9Ri2?h8tN=l2t%7pG*dF@M5T*Ix>Mqac>3?3svlU2vb&u zb!~G4Fe>)L``4Ej&)dg_P-Zt%{q(o)5-Id+_i_!10|%>)CdM*US$>~zX$}Pe#poZB z!e{|pW(MpQfBuQqY~nypuFU^DM2?0X-3ic!FrR3QkGR<)kkXDE9pOT0J!iSljIN&B{e9My4UsPwT9q7yfTLh3pA6ast z3l{+nBaW{&lwS?Yee@O7-5kWg*VoEqSoZ>HxZ;31F{{m5x`;#j(z#`Dxd$G$o4ms0)niEPK8I z>_!?SJnoe)u;x?{5RD<(p-;+I2*sWV1Y1WX=T9fOiJ0$FNGEx>x6cuU zkMWhgS<}LLj0P0;a3O$MibZWO--AYQN`A=4!e(v8lPg{dCn8l~M=vB^o&v*8$OX{b zdTmphzas~0sR>|?6DnZha$`?kRA4+HJU?W8KKaDj?>~O2_exE%8){;Ze%P~&&?w}H zVy`8&<+1b{j}&JU6oj4@2y%VU;wIzR2JgjdGBqb|#{%(LHT4@0X#vK%p^&Fa9^MA- zw&M%7c?Y$^8)TmIu<;{@vaL;(lmj=EoG^F%9QGh_P7FWi2I|dKAZBbSib{p>Z7&fj^yg ze$1uJWF2W1TeG&%J#6iTon?MlHWTGgRGLY1_DwX37D^FbD9|@a)xF(sskYj& zZ?%R_F>B;(56ra7a_s$06}bQlUe=;X#uh?Of(-K8~=8bt8!QV2r65zO4Ef{zAov+W#UW#MEJXLY;uez1hEa!So8_% zB}YnASFkH-l}A=1XP|SO9$k@Bem~N1KeC(>a5mA6<2P`w86fPIQiybO{g{nLo-a<{ zgzi~wMM0T>v5ATGa8pmjJM>7SX++h+^tS$&h|&^F%Ex9I%M;km6lk6*#ln&>E7Vssfz%|bN%0s^M3 zIhuXCc%9@lAAx9r4(D66d2@)H4wYWg;||A4JZUrt8e8zN3Fbcy>9gdcXw+SS(N%;_ zB~xxeqceBglMT>%aDIUhUp%Cs!{?Ezu=>E5^I=D;Axftb!p(G^aC|B25I(L3QResC@6{Mz zvYa|`^pJ+}`cdeRtq%J$#lnzOP_8A%4Zu>eN_MqPBl0c>h9X4a&vq_RqpBqdX#>)k z!H>^Ej_#_QKFUKJbmWcPfGe{tt;lGeN)dtL&|IoLLwmbd;=V~dE^Ml4bd9&J)U<=q z66sb}qjT0--#%J(BI%)eSl6831*kNU!?UE8JK4xb5pY#J{ly?(P3@ajT;|FX`6<$B z$X6d7sq}I1D{ez{BpPE_xP1g0Tt={lq7yvaF7Cjq1En%`f0z-nxAsx$0?sum1Zh&P#6t(tUR*+&hb-iDr3oJ<&NP4JB8>n zPg`W%2&x#jKBg~&4XZ?YRWzy?kN$svCeGws@L=an0mQ2}cNh6vl&u+gca6S6NWfAGkMYBnw@ zx!5K{4#;8Q_~eB(-Md@83>-fIl$cMzm%-NSH{-+JB+eZJ%KYi10DWtYh{uRlteeW)8nDcSNEGRdfa z(>~}<`A+QqAp7dlVoAVFeQmu(4DeRVObW6}ylgdV`rc+*e3=~onqmFVpy2t0B*X3k z*a}f#dwf4y)lrXOPJDg*^()G^oWJ+Vy0r2izsXKl4WqNj0dWJGYK?9m?q-Vg6_fa@ z#d$77|8D+!i2wK68z_wDY8{AvT0PBA1QLHiEOHWmgiH8=y@mMc1A8^z0|Bn{z2y7E zf6}R#6*ap(=b;qFBYX+fxoSPbcZvnfyRkd+(eQ8+IcJ}`asik8Ij>$ss=Q-;mW zNq2NfK(i2BqNQ;5zky3AY%A{rI6_9j2jnT)_ya$}GtrNeRjB^rN6@`!@4S6BiJ#0+ zj8zlxyWoDcbs6Pz9aqB9w{rY~mBXNDff{S(&F)L?q=qnm{$A@N3&r<=RfCqAECt^+ zqLCjgGIzdu*7?hkQc4|#gR$k7C8$eU#Nbt`qObb*`GV^h{eyYb1FL@g z*?bMf@B_b-zm0`25z#x>*LxVuWI8^=T2aRo90^v#r$igQNLjFct)x*7y zeyS0Wn3i*qmGqU5-HN&0QM;8&_YE&{(|fNs?m_zOcfxo+(OKiAsDxKR1^@=HIOtH# zdN`+6ovPGcM0c6_Sbq`9A{YKbdYCB&9Rl4K`=6L&3NzgOJ{ZM(YdTi-iR76073?FE zv(CKzHI)*hn05!Gt^<54B4G}-v+!^)40B2DeprL?CDe^ds23yI;F`m57da;~1Ut1# zLyL?&xQx)ixpmQDRL@@^5}_qBr(CW@n*1^icZ%`}HC_tRxEs3g7w+LRJi4z@EInQQ z+fxKTC2vVJl({*YrtW5^aev6&`$zm1Uxa~kHfH)wI-E=EI1O{JfZ4%-2;QN?-y@9S z=6D~*wX?-nqdyA-Cd^YR3PT&C7HwJpJ)DLl74~YZxFvm$f@*t^CKi)g` zLLTQIlq<{#N*A_5g-)%OkK|d-q#~cAN(C^!u>A=I`I@)|Fu{c)dtZOvp)?IAg1#gY zoLBeKlW*D4ro9zGPfWCyp|8PObv8Rlun$jcmh5mIE{y6=%!B+KSnO%l4|5{b z4o_4VDb(PS*SNQ`5Y+l!_u%`%7rn|kP5`Ao(y>UyP5yPV$Cd8*tC?{#%ehu2!TCfG zJ30m}TwmoW!qd3mj6Z3|Xf*0D`4ol9;ttiaR1AmX=Ll4TkSF%7(4Y1P>OJjpD~y2A zZH@TgSSI*$CE;&)aH#*xKIut;V;X2bG*#9xy=CnG60|iZGnRH|fnTNCB07@0vPLY2 zk0VftaXoSr z+uc)jis(&B$-r!dD>XvZzNbx5zP1(t;NewJ{g0;Z7~9 zm1ONBfs96V^@MVN%r{#3`{6F;lmsj+iV$_JLOA6}ke98(fOF@A3J)Sq5cWAJZrhLATJ?2#9p$)Z_`&)@@< zV0$9+)h4E#1$FT6Er$|}Q#h@-uH|t`+oWZ`;HuPqjIsEMNbdGI>N6!go8yjQ<;&r8 z+k5yEX#Pj%9x%E^logFW|BL8<%c#+pFK`QYDDe5;qtuaVZYsxIO9F1n6 z)7EOfJ$t5!2xSx&b|keK#LTASbeLur914Ei&eg~~(N^;!Nt{AVdzc_J`JA|sM?OqY zbLsNE`x6`*eWKE)u-o~zugsH`-eyGzu8DVBbGoL3el2 z`#${MPDC5x^l|Hb4;GnwdTg+O`i`?x`2&)KUDLcozMB>MHIw9~UJzvxBRk*PgOxYG zE`d&j1mNLw=a9Rnq}Mk1d{)c>d@`2v0|mYVv&5*ks7r?eXF+Qhp`%&5Ixo%AC}9#A zs8mlUUXm=aDbO9_TVvc)=HZn*>vD}@d6dT|A(g~ic2_EkiY6IYb0_rGo2Go347DLd zY(v~Z)3~RiJz4ou^bXzqvt5uflT*F8xnM_-KHCYLK8J(@)JR-pjcw}~%@DIMYuCa) z56XG#P2}j0D`$j%Tx9i;6Lrolf7f4Hyf#EaYkHKW(5IhREitvgp3QkY0s~7V(Ia7{ zm=ow5qobJ6=a3-kaxKrOf{7{RmcgB0h5N8s#)q;<9b-S0{PPD_ z)0|9j=V($CeO`TWj;`4H4KlCR0+K?L?0KhBZ-?`qK`%HbC>1WE^SIr8HptE7F{ZvgD9^wGy>*nj%%^@y z`zB1Jlxo_6!%^?}kST^9Tl29`-&YH6kGN20D1s|So&9@$dL;rbRqS6m>2tIO**M48 zCn?-VV~W+Pw%`}my-$7h64MF#$ zoIOLo*2r)~A*r+MBHMck97sDiF`P}AnYH9N6TR+&c$U}wXl!}1`Ms~s{%U8)*;YqP zVaE3geR(;zLS}Iq(Dj7y7-_#3ku@_~m-eCdW!9;?M=@jY5$dfd!=J3P2X3liQTgA9 zyB_t{8QZ?jG-(s;Klo!>SL^G9am~RoOhnVsSYafRZlt26yz+3N9dhY_sBMEUbu!@L zROu1?U)5R8>>)3KBPK&?vplNJ!DTFjqGdGXz*|zQB|6{+Ud7rvCoW1t$p^`3))(Mv zrBr?ln0-LRD}2-CPsNZ2xuo@g&JAS+bT@=vI*hK7D51em_3Px z(TC$r^n60#h8iQ#xD%IpIle;Ag`%sc^6Bhf84 zx8lvaUZkUzS$pej_!=sCDo2s*D4ci?c}3ONdPwQ+67Tii!86LGXQ7G9;accA`CE+X zeh=U84kRQM$Rcj>mHd=;-o+*jCeQpfv0z(r!X=QVU{y+rng(( z=`yj2A|(~&a9V%?m`qQjP)J{cKwGE2$tF``p?pzi-0O$*jPfP(5wX1Hf$vgu#s|TP z?G;bs#}Hkgba%^HpC^2F;>~YMI12TG< z&8xM9L>3ny|8h?9qX2!Hw%R*nO?pNz{i~UXE1JS8iPlZY4I%J?Ub4uN4 z4BYSZzGqI$`~1H4jvJDD>EM4<1WzOjvsOnK2#VB6 zxHO3hYT%C)+bA^?IN!faCk8}Vokt+j7p5Lb79w9OVK99=#8g+@Eh-902$NgfsF8Jq#qcu1S|7)C%SNfgxev zl`a{+<(y;Jb*dyN%7^`&BamX3FvUrFH|>&yCv=Olq5YmL2EQyW)M3WBN*8Nu^*et+ zG1)!APvnzcx_=sA+of z$j=DWynZ20%eGgZQ|jkZwGv_4)f*dTTDSvGRe_wDMZg$7wb5&I7`AN#_{;*e zJ*7YlAnjyg-Y?~pASNIC*o*(e7K2=aPsjiLxH@#%9P`jrl3 z6m%!Jm{73Z7XFAiht?L!oL%M~$4)>ZYr4hk=Its~WBoN^IrJ5gJk>UNmZp=xhn78^ z%4`{`$e7r&L^WM*NYJVYPE3i+ZqgxCKu`Wu}YDe#>A`Y`Cq1)t#77p7#5Vh&`H%S+}UPt7_oo{~z#lYpqjz?zu z&2{3FiW{U9C1ySGAHATYi=K$HZeQ>*1ki^F3&xxOA%4vww;UAm^${@OC9+aXyiZA- zI~@i~)R-%g09MHVFcU;5o(Ym1_1k>}Y`+Trp*s&4WLzfGq`iK5U7i??z!C~HJh?Ka zr`SPwKu}UlaHM43@_E0*M;Qn|#+PT*`BojEiVYUzwfQdaRk}4;9|I-g1(i_$2?>6V z);7a1q=mLPQePuurS1u}G(V&z)C1mbpIn?UNIu3(5%gX^qo^y_G)r2xgJsI6AZU3d zspSrUbIepbxNqgl8HK*{IFsY+FOBmXh(5N-2AmbS=fmAr@fsG6;oGiu`buT+1P;$m z`b2e;{OS18G0}!<6Tc8PPzlaxu`EZ<2RmX@{m#>dn;kj7__inR0WZzk%T+87-)9Z! zbu-YPhUg;eQ-pH=RTN)~{jQIUHee{GktNNUE5MXag!dtN#vz)SE@xBn1Jl0RWo|M_ z(Ecy;O?9;SHuiaJHTFAtIhDbm*4{0FTz(8=V<+1$=ww{c10K69qUWTd``Zf~#m_E| zB#$%Rl5Nxzk{H^dVOu<1`Yu{j^&9wL1hQCk4bydXVSIARont-Iz2iR4o<))cUDl91#`ia*=A5I$C?j+4pCq=yUF5Uhxg_Kys3gPF(H?Ld zs=)Z0k8xct8VK(QwC!O52r%>BCDTCuIe~1$G8Y^o9Zu+dj*-pO^`OcprL$W4A2Od5 zjQ-O5*wi}##g4x>Jw=&)E;TNzI56ibbIpu$op!(%_6?XV(x6~D(Yf*019k>OgSnw1 z38yh7#04`^PC&qwGQeZ5AdY7<++Z@Ht2-WX?S^on@{!jAb)w{4Wo<#E-Qi>jB^fB_ zpcc1!-BxC+)~vsrHOF4c(xto(Fa-gO=scK0709Byxx0Oe=T$PlWnM4eI+k%4GZf>} zH3|uMX?fQcF}}&gkiH^F5Azt)A_uhIFr60-8OBr6@HVEE2AO-a09RkeCFXgWQhd3l zviq7|0=ZNMAa54LHT{r*5*?#k@-Wi+C9l;PFLEZlQ&tE3p)0m<0^T(V`VL-C0EVYW zf>s4A$Ok(~4k6jRj6Af;wu;R05ey4i=Z4)3#g%l$|1+fnx!!adyK0##tn@CdWLWo& z2JSil+u{+6<97685sQarix&7ZG&H)?&;ZRFc?|;GjO)YpYHjiO#d0GBw;y#O8HpL! zYt4<&=aP5FOFaLLLL4|5S&*aWcE76omJmVKDApk4a?x|t7V5s=nNLcWNS-X!#bOhT zT9r@2u~2>l=q_OXMiGnke6VTgS1lxqSzIa#NbsM@q{@7faO+dez+q;xRu>yRC3r;< zJJx=of-PG?&CXBv}EuhGx*R?R} zm*QE*H5W`=X)X98FEi-El#iE4AljkOiJ$8o2BZp@qpoSr~E`4v##p z<>Qc|MD%2?`{l8f)ZIE^&xv4>Z!()4&SHiP_ZWL??DhCh*mVeZg*dR9_{FSEeNZ~+ zJReejuRYD`AlKhu@2t=3vn)xrh{SgO{bS{YI+@v^g9kKVu|o+ND=cXqXL4vN_V1T3 zvYOHBli8Joc&8p3H5kyTzEawpRFzg)wv|ZxeBhYIDGG8$J7RMZKXhSeKsZ|Yl44=R zGs0~kaUJK8jp8(i(f9ol0;7PM@0LQx{8^MxtUHE@x;uJ>;?AL0bE0qJ+A)QO=geN@ zQeI3-OUcFhMfR0OyL>}qZ7aJi{3AZ2aQI;^lRf8|BvTouMNl)_^KdSRwjuOwF~zOXec8pCesP#Vo&2H4n>s$&9Tr&1LG|5;Cqf44MKS_=Qa) z+ndaS4?Ruryn|bLdT<8B&t(d}P}_Xt3hkTqGxXk_iphhkEc<-y!i%4w6l&B^c-zOQ zJ-YYBm{fQUvvAT-J)w|`r%k|F-HPv*>r=FR;p)HW^k7nZ=ej;>*8~UUC%XU|(nm^F zLaPOtoFK}g?0)wi($h)Dl=6EmHa!=%3AU_VZ5!%r#<80I0HeH!Guwf|UXJ=6cp0^~ z9Bf^RA}{7Gs>5@{E@!1}F?W0v_PKf0oPaXh+?UvP`U{d$IAmk~LTY?3-1GQ`mufq2 z?UJQ+Y<}90Ne8!Bd@7>hMg`^XL=*>~$0zE!M}J%I82IqhznBt}Srk*pStkMIxSdGb z5h(_tePC{59d?7QytHF==TJAll73=!-$$eif_I4TH*Q`+L5@=-^dX%?n7hGwB)pNq zR6~)zz_BgqBqp+mr#_ok4gtJAX`goVbBtcJ+Z{Q#FHg6j^W`RPzDBEAC{_O3Y1)Sv zhj^uUy?c<}W`3r>yRr4pKLsk0cuVhk4$gHprYymr(sZY+d?Y&Squ@~@R}KLwR=QHj zN@IR0OlFn!n+_MTvFQvsx_k4$ecup^Zqzf3_o^LM$~4J-I9rpQ$ZF1U$_a%*i#uq& zX<7N~bk?{1dockzSv(R-=feRyC=RYe(tHBu-;~egT6J&+b{4R{iX%#a+pF{8Z8b@< zend~&w60{GhmR5=Zf?aW&hn{@BTQ@X{P#1r4{KTE#+h7uW-eiJ7H(@>zp`?=dH-AH zD}$JMKw6gRrNh(=eDeICo`V2~&G=orlME1KgXR3LWvP*Y zUzk4*6>|I~Y*qxwL-fY*Tqbb2whKDc@LFDvT7*kE=y0YoNr?Is%FR>YI`fl+vV7h) zXpFbRS>nDmBhT4aR=u6G#RKJN(wit5Rmt-BjzsgDL*aP)_aCVeTy_`(hqe60wQL4h zU^z|f0+Pa=rO^mJc$bZqXFLu(mierJT8dI~BN!P>u6PpdJivpT;WYT5sqLI#Y~v)n z9ULr$X7zl|_pI@%0#>%UaR|?{aX-1@!JTJT?jx zcGH!{+QuPTA8D%K6L6Q_-Kr$9;|i)GxWjQH4(!FeujTL5_uGznZ-Z~$FufCU-DU2> ztfKk}%*9FA;i89SufkM5N)*<=hbHg~_FB~LHoJ3Mu3NVg<>E`I*vRkEhWpden8P!h zx6~_{Ez5|5iqrwG|pk)?tjw|<#vY=&=*D{;wG*W!We*%y`-e- z7{KxI(&8hLI%BT-O!COcyuJXI-SKvr?T%TkR=2! zIIrwIE(M=pBUg~fMCj}EKN)D-(16K*@l>}j43q)zb9{N<-j_TEz&GJMVs34AuAJ|S z1qI%lJ!k)W_ji4=g?pPq{28vr$<^7fLR)PS9To!YSyx7Yt!w=MspEa17wG+8lFQ0W zxu3?I3yhUoJuD0e_7anY*`Dwz&As}%ix5fJz-@vcM5ndFYHiC>#v zuO*UYy8ABCliD_n(M)k7`)A>h;bxKy+?#BeFWU9q<(3N%x8UDrqn|> zoMz3r0Tym$3sK&Y1@4`4BK0cIH-b$923#EYQU2@w#iLmVyD08oRhv5gIw2UHWHp)e zRQ-BxO7-}nC)Ze#t+X;(XjUti8CCwWBWpoBJGQSAGIIxzc{NV?}W~e zpUugAR7AJ8XDYD~-89%hwvFK`dCy;IB?f=4e*Dr!WO=~tR>5L~*n{&sZjs^kl>Y~- z2-={rnrVTwH<_iWjICxzgfeM+rlo{|4HowXiliu3!tZ)(n9aX2V6bap)Y{w_sx9%F zTP$2L7ERpF=|0ZN1FqwfHr&Dwp0_kN1P&G>wa&b9bnk$WFh@HJNy+EF{JOk9m*4N8 z6>seYN+_+OOz#sqd4+IOFely4{(MwoaE3?t$j0Qzuu*VA4W(SX6)iw%tArr0o7+qL zZ4G4QM|V&dL^uwkpxrR$42V-^RUK58xJ!%(rmqJDbSg74>Oku15EmQR%s~JGC4f!4 z!n%sVd8V2&R5SxZY#R;Sh*V}&NL#pawGsOjYomg@Ou8Dx_uF!1!NJV|VDsh%MCvZ6 zA6urmRUwuj!ivrjR!SgxX%Or*1*l5Kf^M!cm^aXRRJoh2Wi+x7;01tDL;c7 zUiiSrSo}!Wh|t7|o@%SsmW=-&RBr+*X=4r^;^IM|^N{ulpVaXo$-T7yfV!S35Wu7a zK%RJm?mCe6`>R$F#Q$mZEM+WK7^4|rM%p6MdZw=Idm`JOc!nn^13=BiHBe85f|vo< zr0eFBm4!+iUXRo7%Cw7B7HZ8ZD+?l#N&sa{@Eek^#<~X33)jyv{jS&cztz)L2KX?% z%`I0R)GyySDs1rJiH_5;%cp`z!ofPPIdFpkctYFB~o1gC2p0zD*`@ z%oYeR@$N4x`xUghppHU1q}F=bRr=dJ>1QH+tp8*I1T_#8htMMV!Ud?H^pEJ%KvDw4 zwkSX+J^jgJYqJjuvLv!^B4uR2ZNqaVw^z9`>!4;)1<){M)lvINY&>IgUx{OABclhe z4e*|D6_S=By`OnK(YX#GcB-DPXCgoNa^e|vIiLx$jZgw8^c9`oO_yZYrhZ1Y1UsCF zrdISOq}FBfdQ|K3dSs#`&plZJZL!lKLp5cOq106^X8U9^z2ij{LgGd~H@!fbqKvHT`DBT^1qTioRr?Ad- zXt7OV^x30GRVPcS!Q3F}7v!W^fwA%p|D#6F2|566JCp&j%?hsza-Q$6@(tuZ3cVkk zvy$mdtz{HR!Y`&ZRoBD*-^q3()Rv) z9j&7N=R`qh-kxQl0`&1K(6-^tK$DCQOQeO`f9q#XR)Eg({?B7dVlT^kPILM~5)iK7 zko{oCL`yujXW6K{-8>oVDx}`(cvGm?fv>5-7bW44SV7Au_Vz#PFSxbfgtD~YnI$gX z?%%^4ODfCw8<_b@+((M0VrRxyF>VTLSfVveD1DT(8Eddx8xuv(`_7}sd6Rw_;*tQG zp`M`8yeDjT^UxE}@Jqc3h^0|4`?K4YRCmh4`f-ZvPs?1Y+E{ za&u0JSD!sy9$(G@iV@3p#2USRyd`j|N=mvH1WGb2YiUu24IFrRU(LU;4-K?{b}+RZ zm1I&^(^x)*uEMFOt%E{W&C6@X=x%2OI&5z44j@}fXAkU55tD;meBCW$m92h)mvUEA z5r1U$QBGYklNp?q8e8pbn8fx8hx- za@UJANZeI-1IXE*k&aXtWC##|L4{6cXE_d&FLQn+0_0zKTCWLVWCU{@g#^PLtm%2q zbwhD6%ZKHQZH3C~Xt!7Y3P0!9dFEiU-Ef(?m~TDNH)h3BMiJPb9s=Q7>WM+1rJ=ab zMhLQYsm%uqS&g;N0kexc=t@ZM_0q0+jB3^w$>{YL5_tLzeywzhxo9#)VY!={pFMb) z%MvuSHgQ;1IVi< z%6Cx0?2KI0XZhu3H=-aySL25({W0v<%3Fmkv?0`zgOJ$c8D|`jl#{vbk+Kq5dxyzz zU6q8xMnZKC7jH?st(onagHpO8$$SVG2Md3!fH~*N3LBiu&^Rf{b1~{v%>!MDAvVc# zC2H7H#tNJH$*sf$>d=RfiUJa8QGJCqPIXu`qTLiZ$KEjQ)3cyX9CCJx_JU!3O_3-T<_hn`joc*N*x=4qc>K$dzD~!^L-D^3?K_5o5XFh;_6wvJ5Jfao`Ebh* zjV#lA2bF1)%yw#@*ZGP+P;a{UhC0M%Imgtj(}3{a<8%w>MLlE(mHXv`23k%5sj0VT z8Z)WMl4{}ng?Okitg!bOZ}h-3us@@ZRT_a9b9|HkuAdE-E-&OHzrfP5nd;XOY*2Iv zjeaDf-4}e$w1*C9Hy-BmC+3`X$9^yy#(ciGOKFA~T>O8o-d{wBllUuQ16$BD9nyVe z;<-Ncdb4bsQ|#>~acf7X*VeDEz8o3nT*>b_dWex5tj(K5WZm#F3b3fF?*0Y9$j z{XR*{4*`~>(ZiyE0K1WV-%Nv7gTMn;KS75P7YFI9@ohEbEj3=Re-z=s-LBp)Fp-pC zs04lIq>5UZ(=+|}NC(+rl(fmAS|HVwZuiq?l~k3K8TiTew>7ezEu%%hJaT?6@;eFg z{UwN%M*`L#h?PDG(F9N}k+cpQ`wGu5>pWwRLr4PdT zO*xv2InvB^wf{B|iY2Aev;44Ys2h(iX^5kHOgk~u_nzAt2WI~jpe|uJHwNj`G8yl?9#k!$su-Y z$oO**cK!|p0*fKPFC%23n1qF)$*8c(o|g9>aV$>g>eIa$ZkP%$O8b?dtJE3D^hJOP z-hq8se0gvrp!So}AF?-%MYA!H^HE0G*k`^O3ISZ{$uX0gS`xL(rd4pEJakcA$S7-Da*0LHZ*`kI@RwbkCB#UQB{ z>V>y3fPhBMrctsH8!Mxz2@1VzRFBAE zzn2i830v=LhU$wc18mBa*9C1sfET(HZ+e1#CqU$8QB#i<>&4q^QJe%nuj#Opgf|yz zyZt|0=`id^g+s3Pn5?j3&3QhAE=%QkPgNAwYRF@05JKc}@O)15aV=lWkh6d%w!oC> zLWmr(<(JR&oH~rfHD$ZN!sIlE#!nuAR#Di>6C+B8H$sX)afTDnwnZ7h*d=50MkNQ{ z*ZLG_c{&Wkf1Yku0%WniwHJvHRy((s{k_z0!$gh6tsbjxpUYbmr8JX={co_ag2^)f z2GJuIhf2MuWUlz3Rh4O>S|J#8|8aG}0TX?UYZ=!+hPs+tT}!Wg-dC7HbD@)9dV>Ta z7{&?J*LEm3$^d^~&Z|3I5SdpxN!wh1vR7;Nvj8>EC-s}FH4EiFd&+^z@9_5rbrlGS z8X^H%ZHEcUP%KVH9?1+8-&1ptSOe47OmmQ)5n?gEe&!SOPLE#cEgS$HTq+r>{P~## z@*`Eo#^;TGK41Bde#VYwDLX;~jE)zbo5zuMO8kfU*T`2PbI8g>+T*;%&tKI6^h$tL zzvL$g?Ae*G$-qJf;8;f5!uf7psQPiIi-JASb)Bx9_(}L)Ai3h>Ih+jx5}m#NNk@~z4~#8ApcQf_G5_PFogS*;x(D@dEcz=%S1BJ` zec*rX=rlshxoXV6CuXC1%-c&x57c8EnF<+u7Xx#hV#1olK}kX~hUreaWxH)WimnUlzVwe$XRG_6tk%M|un(Ks zW@35jsD&VUaA`i%DI~Wc9D{wByMFmG6s&RJ_Hl&P_BN_-q_#ePWS4sM%R2H2&RRxR zFcF+T4xm9=ATDN71nl0dR3ItB)f5b$1NncS%Z098pr`_QVR8qXd$Q~IVqr2=%is1p ztx|{3DP%;1X@uomS&UmLr?XDP@XExBci7=f4z`Gn|Mq{SCtlMVnruc@c0XoDconDqOnYZ2AI{h4~&FpzI+23-}e z@^ZJ*m7NQ`BRk6psk%^1QdCK>md%U(Y8iuYB2C@XGj_{DIn`PSWn1f1#nAnQIfnl( z)ykA3_qnDpLoO0db6c4W=3>FsFfSqB8j4vml#DGGpCTRj?guM>mW&A0#TnCz_Gk1- z8F_1Oupmd;#B{LmI=Fgi!=_@q%Qe{58fkEdF5InyP83SjZNyKTPvGd)4&>ll0mSc- z)ucWbjgOY0h{aP08?j;&!Xc@ChZ*QRI}N>;ts)f!P94N0?=fy~!Q(kb93R<<3Acu7 zbpda?gmb$^M-0)~1=Jf+k9++YUW`{lp4`gVki*M<3N8wm$N`1LcQHTnu_tI~#K^OW z`Yi3QtQ%*DXwY69*SC0nPGz{9C^>}BOJ0A{__F%G zg?dzf&j5yC)c}5Ra-uv2Wd1w+B63#pXNnu&dj1~Jh4K&uz`m~aGL9FIqv*RPP zRVh5SeAkExGyb>zS7>2v;~7A|hbHF2RXNQ)r&c~j+BP3u?PqoC-GE^DvQ;Qw&D%31 z!G?q)0@lOPrJ~owT||9rn03Im8y4DfBH(2&;^2Lh+MfN6XvKh2l4pY-4{Yvsj#YSE z7VFc%!G5~lLJYD#$^^x6eB_K;#V=FxO*7f26gnZ}sS(ma9Qju5`8(zx;xYMdvb{)} z7yIWyZVAy9``n;&+_Vp6MDh!^)z|HPF1?FtH}IK+{_5!Y6b$Ia}piL zN`F!D##3}hyMZ_{wD>{B(LDi7n&A#Vv2dE<+1)H})ihMRC4$3mt>R=hHf~=yFGA;& zMi+PDmSY@D^JbL(L0gVQghZKud!e9NDks=%sL$VxW;vITbZ|5HZiuXOEjHVz2S2{& zjGqAmDSJ!!P$Os;RXel^sCIJZ!kWbuqRn%-!px}1u2b2Y67HtQ&ReQ+5DJPOM13RE zcNR2zNbb$W77!m|LGPQ{vIsZDP=Yu1u8=E&68X*zIYYQK)($r0R9r8T5@GEYH2=-j zg^v38H~brVr6F+ouF>u6i}>pegWDNIcR8QL9L0iU|Ajs)V7_!)MeGGYaxt!&1RQRR zTPW_fnjg1WP#xUF;w~_KFMJLvt6}#pH&$t*B=?=&Io2CbWBwGFZ2ihz&SGQxNdA9> zZtl7XCZx{dE|Nq7A*4(rDR0$U2mfYK(kM?hHJ84|i6PCHn z!Ul7?8Hr&3wi_@9{%_7U`&%UR7P*~7!Z$;&Wk}tVg_ngn#1h;XXblj8MTykfW@l;b1R#xL0Zr`l-ib>%gj?CEThg zU9vhrCGX-WV-UcslBFswSUqmeq%Cq$e`i0jvjRo)TqYV$X#7W$7#kl$z;WdMNB{J} z!VOO2%T?bofh%vR@y9g-JVObD+@KFz(pA4SoMhj^01^Z4TRrt6<{El?-QVz$$0B86 zg!@e@C>=4=?+GGrlB2V5u8IQP`93)ZOB7;r$;s^Xf88PN^w8R}Y(fxpxcU%hqq*9! zqt4F-2`N%C*!EA9iwtgA?V$?Lk^K>r>4vfxLbwdEe|cH=a6PsBFe!NRwainB^dg+M zhRtB^_fOEPPhjUKZhtmm{8D@ffr|1BL6wUD=(#UzJ5<0r0L;Sr1qy@dw7=_8#wxfiwf0DeP|?xnR;a;)vJJMg&PN)K zV_AGg`Q04nHTWoq2*`P)Tr;v#B2Ft*D)|Sm(vn&#w9+?GL)~^*g94g0ED?NOhL;uu z_lK46Iu+-0oC!NM{^%MW`imvw7(lp!d@8~h2qR5Ng)nq$H zmxWr}+J6fh8ajk#;fkI2%JA`*L{jx^rOWb*+9Yjo0rl)oJW-Ozvy#^_K1|V7jT20R zTwm{}H1So zZwcwCd_1x%Q}&0k4Y4@kE)*_1A$+x(31GCwQ zUC8Z$(v=(*s2;TMiU|3GVK~Y-_e%590=ufJ8eA`&5-`sB0TwQq7MA*yuehOOG_I)1 zIL%bHX5;pIVwc)(2ESP)ti8tA);1ik_O!sME0;p?f4Y2ET%& zy*^uq(COu2;l#`oPot#myzvU6z2PnS>(!c$_yEHmoRv<05PUm7F9giO4q$6c$C};kR4RlQnEQtXGUGF=s~)H zGwyWDp{)^pFS5U9HWERgK!bBbp!fGI5#!VsSmIS@PpHe8i;Rq+FE~EW_WPM=mJn}h z68e@Q*Oz_Yg@zS_?LIQ((>9I%9_hB)U*eey>PH)qQTe5Zbl%;^5AiuF6)8;8;5MZ3 zk^#Yt{$9Y&iq_fd2^$fl&J%wS;-~&t@&zRS2dD}vPp0#z&hthpf%+xbpLhztVfeGx z=-xcXdIwN5rYY!+Bh}PsU-+v1TWA$R>=1oDU0G2}xX?MU3OO&7Id*9%V~M=0VB`hQ z{l|IgSC+W-dh z7WQ}4k$$*<1l|?xIsz8G{bxi zr*^A-Q=*n@H@b?ieMx=ga}=f4Ny_V7up*e-v8oNAK_$~avyG+TEvd>B)E=*ht>>TB z^u(*lM*H(k@{CdfOwwew8E!oh)`@9R2;~yjK9(MY?+7$Cn%SCpEao}Bg%9*i=muY2 zT-e^TCm5B6Th9wF32wW8kb9x{61l+_JLPi8yTV%>)>E+3I*u!UdZTFGJhis4T2eK< z!JtuA!^O4iR4|Ovj?`N+MZ7%ph`WKlqx^HAX+mopFtfn1=Dsy^U+X<1o0fQQ->M#Q zn`1D#<2IaExl3a3!{$6oAoQv}ZtPi_GbjdhsKV=M&*zU(3dQ85_4Z}ttcOxN;s^!2 zLTdUENU_`M4y|CUAbu4(3?~=Slup8fu+@uco-7H2G{MP64v#n|Gr8bp#?|D);B~n_ zWQvF-)Ah>CMu26GpAPpf>3(W}{h0hs7Q6L?WV~{UH zSnE*MvkxSEXDvkr=)5!nA9YwoqjsD3PZ2A@AnTBKgQO*hLjpro@0Vte_lr!Z*;^(A zYGeACJZ*T{BP}d#W95W@^k%m{R$v|=Xkna4Gzo}|@Rp`*{vTOy8P!%7t&J9Fpm>YB zLvf0`wKxO`?oixai@R%}SRuG;aVf6F-Q6kf{)P9PGsYd`{(!x<0LjiunCr2*D;5Vh zNh7BB5BQryIkehtuZ1ahLhHMrQ`boQUdn*8vmDXTe>HX8yLrF$7DxRAhXN4RX)}2D z+&ux?BzQXz#VV@f+pwi%=5tV!9+pUfYMX!H*u}8>GKMjhT*;St8v!hd7*>W8k3rwA zZ2IGchOkQq?$&L9$wUlfaH3GovYB8=AO0nx$0(}|X-`3#1B`<|G+88Fbks@@(?LoDz<^iJGoZU_p zl>I8?CdkpjzB`5l)9H~cdXia)tr*Zx5AVJlcD?jLn_I-;F3kS@5KxTlw`giPNv;>c zg$;Q|np8Y#WD}oawurEv0JX=ZUR+n^ZcpdXr8#Tmhl9zFp@{Y52ZnQ>lfA-1#X1tn zu+h#29o4pZNpv)obg~lR-OD81l;R;piPzcxERswD=&XbH6ANC^scyc7(8P8WGx(-z zQLi=%?qh8MzFhg2(32Ca2{bA!kR|kfhRIAX6na5xzeoIFC7LFmsaANVeS>`xo8bxX z&M#xPj2)ux%16H4?5y)|NZoU5^!s^PZ2-9ownbz4HO1H>FLO5WJJ^P=LvZDc{dlo* z(ki~_?XmqqVrrtRe-Dt|h!|oy^us|-&QR*YIl;uP^!teUug4?I*&~SH`xqyQY$YXtV<*zLoH(NwXy` zw?Ty$cjfXwyYCF1A!ex_m#U|oYdoKV_9a7pa&&Y6e(25#r^Zu<)tY}_c=*OwAJ|7T z*?=hcOc*PJo9ZsXQ+MFKrBbwnTnMmrl)QY|Pd(CCZ)#tsyjV-f>fXhSCUTy0$X;j=Ye>$`FCcR{fkgD@jv-)BR`;1;>Cpk;)ve=i#I*`H*0m|M#F(vO#Sb7`FrObeonp^RP&U7;3@0PH7yvYy$j9i7(^Of(j9sPx|aIFEs#s$eb8&&`rSu!Sp6ca2kbd zJBsY2B1Fml!C%N}d_QckSo5jt0H&Vz+ZUWd-CSJxg=bu5$4*7l7$J$kEm)3%uo3(V zcHVD3IE5oG$Y!(wgBFPfPC4FRnfLs@^)wWnN{`~LMP4#KFS?DUbkN)i*(UrTMeo#n zFoSmCA#b`pkAqS=6F$SpZU|UG1SFADTUL%CnTv0-tYe?y^~%w5pZB9HCw?zN-{9nLYhpLis2Ul~*V3=7v9 zFVAAo=0rC=VT@e@TCq2yTk_2v{adi6EhrL6jX!SNZqesQA;ShvG1c-O=z-D@Ta&0J zWqSMC&7E#tTb)MM!K%Jn^T-c_s%8#s%$YH@lqX_*1f&xd*nQ(o)(&nO`gpkA81+9!eG6kGys})1Yg}Kv0Eei*xr?~1pdNyb!I}~vW_(~ zbIIwwxq}RxP0YeRr!%=J)MWEbqh89f|0dx!Tsu=JLju{i+%*zRXP=9aO(FnB-#QC!Y)d-reD3=%%dqH ziV4;cg@Zzu$c8ccc_fk>lL}cs$VwEs7m42J&@oqGBg=hFdG`s|)URcWp;VzvIhkm? zp}aJl^AMfOwYEl2r8mt*|Bka{Nxy6rHuzUB`2$U&5gp@c`dGUZKHi+pSjupWij?RIyns`?dQ(DJ%$b3~^WgKpo|(Nb{3AcDY@^o^)Dn2L%xzSbMXCxWUw7GdY=J8RsYAx0meN zeZ~co$;)vO+f;n1#s|!_uO|!UWDUzG+iGHPO`H=Vwi^TkOuEq-sZD4c7r80gnzYIl zVy-{;O|S$>@_UK}g3nR~jy|GDPH|H$W^he`PdV@!{%kSt&LZ|2bH#W0h~Z*8r_)4+ zy9FX8Xv@;9B=i|>0$7PNvQ^w62g0jPQcdsRosLpEiE>-I=)JCxzCW~*>_!haHK6yl zgfIjXf|w?Ul{BoyNQOxN$@abug*z}sU zsPSTg>IVYAoHb-7v5ZohRB(JD?ttf;S4mMQxDtoq51{*`5og&5pfj=*9m*hx^96s1 z2$>Er5p8i}DERPWl@MfPCMxqDCr`{kWT?cl$O54+2#ljKjw7luUhWKhB=217vR=ey zCFksb-?n`n399{oB2`=oD8{!I)j~#*vhs&{rYOw)b0k5>A;_#3T?HuiuoYcAW+OXx z$B%POLy>9$5`b)518yR7?8Zr2uX6-lj|z)^T*>X9WFoGt{3e6)F6QW~G0ctx%?M*b zC!Ors+JAvX4v8BQeyA^+q!RaU*OCZgf&ywFOhv@V9}@~xUKynhpifQk3t4_Af#afF z`nX^?@P7_fK~5bVLy`&OYIbPyraJ=VeV}R`rYd{D#k;OVv}ScuAS zGD(Fm2hpSJUM4 zS%55~po*w?>0bb&tjig~%f68q>0&F?#O5(&cF-}MjIMnOxEYO&&LzjE#;s#5(dC;I+sUd5Y$QnQVT>1@kxF2kyzI@_;>_PUa-*`)iOaXD>=3Q zm}LI`{ZAVF=nYfN2-WzV(m|6|v^Z59%JMi^#Wt{ihu?3}%m-QyxTdlx40>cN2`_Jc z|NhKNPg~NU{p5-V5Ub9P%J*%wV>{cRaQwTf^<7z;*%>zyg+HG|r?im1QJam+MgE%- zE{`|un$4lIM-sowXKBXA6rjz$l3><&wN6|n>#rWpP0z(xgBS%Hd<`G5u7w|>4aJmb z?+b8L*6E7O(lq~^c#RPB7xmF{ky`%KM+VO>ZQukJ)^2leNl{8GeOOhJA7h*ZWTj5^ zMmDieV*hk^3P(Ns@bS$11nlf&{rT4e4L(Z*5onVF-(mZ!(5}#!zf1Ku?=vU92hEhx zTn!rPIeGsYqZ|#=tyVk^1p}WCWfb@f!n8Rs4tNn2Fq*qjBVR9JFKCtPM;_vNMM-_& zP7e@U|MFyeW-?2OsulQJ#0ASB-?|#hlWeX20J>Pm-SChRJ{NMV%9*UMHL_1`qkeT! zOe;H5$YRtHk=NUMCzh}XLxXlgPG!=A?4|A?Jgu4|@iMs8-hq%)=vm>?B>pbx-suFZaRQ5bgu`JCIF z2_m;C+%+LVTwd5KKZ|0he|L6N?5`;1hA5)G|KOnpQxZ`}=CX7(w>PIR@BVp)@K^JW z0U+9a8NO^Oi;^?ko8~jvU>9F>OzSN4;3m|nq=IMNYiC*0Ny({6dmunv>*b*Rm_oI4 zxA_ zehDd3Nw&idoq1|zR~OfNfelfy05w&i0zSrlRIE?=wsAfQz_@=yeM(*8r~3v(ju@$q zv4_{A*9tG$_1@g-Hee3Q*WU2y>b>z>QNQ=Iv3{*huUXX%OEV@jGuO67oW*yc-NbZ5 zY8WGRb@DmhuiO}V%6`>AJL>3SR*uK1dn6-5xIXt319h2U^d+V9?a0)Ko&ERvfs=4L zqDj^+a1DEbhK2^`lVVwtXDiJ#|33T)%}ih`%n?a)cY#yAm;o5y=TY*U84X%!QY3zS18*LyaZbe| z0Xx5gRJ`PofcuM+D8_s)J2IFsm&6JaZo1+b-gA}0B;OT*r$e{;Y2d>G(;|c`4Y{b- z2_{<440F1o3x1}40-IyoJaBPOuOp8G_j2_eCW1-5Uq4I{VCeb=TV>V-&SX~v;Ws=n zd)jlfgjohF^@>QoAa|8adMDo~Y!XPP!^w>fW1hxcMSJE~lb=-?+{#OD&KRW+O>+5O z>~`GDK47_&z8M!m;p9h8&?IFHlWQzTJlj&w5k7p8iE*HHECU zJ$4X+bL1kfxC%%bnJq*u3b}|TjPOo1W>3OKto$JO!R$1oY^VnFSx|GLER)(6azVsx zZx;jnsEq+%gzyA=2%`>Jqc&~#^CW`W=lhY1W|rYm^IQEqjhP`NsmOP#0wHPo9b;`T zUuIY3%$oxAdyRN5k?&l}T&5l2+HbUe%`QR?23gj*^Uw5c`Ub+A0@iwrJfD#7MqpA% zUSWNbayJH8Ag#Kmwqky0ht-!LhC`XAa)A}*gX)O=9t@EW1cbMX>ViNKvJU(1a z2p8{?URFi&+qo4FP65jg={WOp&=IPN{4u;`Tce*=%)Ak+NL{0>_6WM7RWRz|msj|q z$QU8jZ*);0e04>=niI(Rl)$}NUA-~4NmIa3<#I>KpZFVIjv~oE_`0n& ze+e_|@%<=NOQ$x!zdCzg`Gh;)CpE2#P?KQFNtippC1J0Mcy z12t|>F$kcBIV~KWD-Ne4tcz=2U*x#pbrW(**T;7OZ;9F6Xw5~Z6)=q?u`1$X7q`(woE;^R0A$bd^btwKX~4ROSMLaWTQ!Ax_^w-Q@p zNnBEG`?~!VJ=genXyC+`uM}B|Okut0-3|-n1yC#Or*8EMbJy;7hWqgqHkpW-)8-dA z5H^npYrIE+3dWrd0U*UoD>5Vs6H2uF&be4^B7k}3fyDsv(4`0nE35b$Ni!*E8V|8m zv_UOr|RHwxf}d*5s<|Div=eh24cBk1^CtwUH@|XyO-f z_I15emSTWqBq6K@bwRshDfP!vr%44Eam3-Bh%6g|Rn_c4bcWZ+)`>nlqJ}aaWc&+o=(uqylxv zUp~hMX@^L+a2MqeLseI&HAUr>h9Mc97IGtKT+}-z%8q=sbQYFe)Y$WR^=Oq_T~x(G zS9T#oVQtuN1dSt^jSN~m5UReKcI|1L5AxDmTomW4MGp=H{jionZIq@nOaUMINf*O# z({Zlj?Ci9fC{NlgOcx53%=Z-tNq;T|cW_qYUy|&fp1bWR88npUxQ3pIXm-*qC_+MEDrJA-) zKqav>`OZ*Fw!8*-)*Q&&otQoyD#X{lmJZd8B|ea`nC-WTTlm_6NS(Xzxr{+s+xge4 zc|!X^>`!gd-U)NKgpOHPtN1U_V+TENJJti7SfVB}Tyc%k!YoXj5oHk3rqh2Pcd(OI)az-8F+Ax| zLm<=kd}c!_agad+0|ybi3i1XQrdPY3viUu#5fUI?+u*Bin%?IsZIwhYflT7i*Jq<4 zco!5Uin32?XmY-PL@kKf=>)}a@xe$IZ@t{N<3Pb@eN7*5%Sh$n=EeQpHPtgg`b6V{8B z^c;_r`Y#8d*7H%CqFmY!5r3~=f`=Gv53tn_^h>Jy_@V*~RJxg*Cx}eOz|srT33@qL zM4=Rx@r|xnxJHr8+!5w=6+Gn6J)a%r^Gj2vx$s^U55!b_q*e04GZE0wl{4EJ?Xsv6 z>fYQN{ND)R*BmxpG=}s)#yfQicjx@hF%kaRaln7=|6=(pO1nQrK8aW$p*w8QCf1i1 zDy1P>VkluZmbTJJ12JbSDEc?!HRYbrI9xF1~EbENm_&>G2R zGhp7E@j`2gz*nWTQCUz>#;;sbEF z3TkXTYrGXm0?rOmN2GW$YbFWBc~d>tS<+_Jb&RT}_Ozk+yy|ziHYa7FgX}!8yTNj& zFLf{lTcN5r2_wbk{oC}n4;5F!_e>jmxSp||+V&pTMN&9iD}`apg-Ep-hSxI=UmVZy ziBKTV;6C2^i7DT3x$s&FN*g#;?%1zpsj)fwI;6k6MZIyEDVbxGsF;;(eESCN!yV$C zHb&38bm?tk99rvBg>Tdcu+ghZ6}G-_A4*Ma{!PkF+f8-bNHu5{Z|G_=_9f$CHm^%) zwb%aZBi-&}2-99%I$|F;Q3ML9UTxMeU7C0{t|v7!Hd^NSwPE5kbfDWSyWhQPznJ?y1-ucR;!)^>kJ? zg^G1`-c?fos*h?!EZ0`0kWQd}|3f65yIS}^xI4oS&#BTExOdrenW_|z?|ks^?P!v8 zIx9Etb2k3>h?Or*&J(}~dkmNKWXu`d-ci~1^54na=qg-B@xTJ~c0VmT=5HxnGywaJS?K^tsltcU;xf**7VWp#aX#j%l|f_v=v_ z)r1pN6^~|aMH6?==?Cr(N_;E75HI(DQFv5TB#Fr*R1Q9)K-onIzqR_`$#BmFMsHZQI#OMv9sk(>cgBcrN{sBXzT6O4PnGdo@*i6gi~nm@C-B_-auWWdVLXImOsv0LPz+_o$y; zVCqIR4C-t5U@h|-{Vr$%3bjuOIA})hG{|Jr^`U=OAo{zeA&0M)37Sild0e5ObyX^a zs{*;Ge-S>Kb_ZHxTYT%b89!^pRpg&UzB?`~<+RZ2uxqPGPORySh5acx1)VQ9i)UE4 zXXzcKf6%uD1?6Z&e70L!HQAy{&#wXi1ZaMwyUYxEGhq1GflQL>t zu&f)FoYwCdki1-r9(|!!Y(o`_^mC?{D;o)za~85^09e>>V*p03n6x8`O%oX@4$327 z31^pN59-ZhFT4N-Y!}})wZiW{?--~XKjM-BOw^6xzOd6!S8bnJ{`o2_YV+a*(5f zFtsi1>{Zjm@^UP`BOWJxFljmDv-x&G<=$FtHEm(ko`Y!dGmQs@z5RXZS8$3&2nRMF z1Fb`ngsJ)*k?}{z5IW<$e}6~U4NLmSO?q+iP!7 z=J+hIm4YbSN7&tKX3J1>bWopzoHLeS+;vz`o)Ypc zpv!$OQ@7VN;a*LVmM-TiE}wNPVbeuPM1p^VP91q{=-CMoIWp+@1OiALcQUS^QlMXM zIx5Lv=XEaXT!!?;YFa@r#EMF`j=DgjH3dv`C2^2t57c+sDDJ_zNt6HeePV(?m7^Op z7HG5pTJwREgGA}Ltreq#MNd`In|&w$OCV~t1BnkYm<58oIo>o_J5V_C%`SZawNBL7 zK_if$zGtZCBC)krA{|U(6A2RP$P*fr1=?|4m&b<6_cT}Gv3J)_sZYy%d`WCZ$%8P% z(G|&l2Dm6aB~Fem=e_M3k$s!jupnxQ%7fU#@)v7=Mz?&t(DW8g`fgxJ1(cB<&P(Z8 z&MC>exr%NJo@Ld+{ijh*q-<#jQoX^DzDH8lg2piQhhtvi&|r5LqF zlO32??5(94tB|{plpW%M$17CyQRo#vbnOq;DF~e=!?Q3bnZqlPS<1!l3=>BmH*WZ92A28!ESD{SP_j# z;K#w^Yd8a9?7Z2gvLuj z{q!LNdD9L-|%g&&Ya5t}ept4iC^zo03q`J_vM{lbSkQlVo z(u|B`zk0|_;yOz@%8Eh%r+-)xeitR_(3H=wJA2B~zv==gIol7gAwCk8G>bx$6!FHv zTwhHRaU}E4cdO(?ECKTqE#Kt+^`eb}K+7huoUvVb4>2Ux1snBSVkojLQ{ zAu^e(Z)b7YhPn594_x-gFjt|tWu?Yc#Yg6jI~JND2WHtHpafJ~Yfk$k~u$H9Z$)p;I_#H?6`++ZLXS$j^_TmnU^L$MLj0Nd*7slaO#P z@tyF3SUq7fQbvL9GABbagy}nNw#-ji-Os!oC8U`|`H~B_4!nhp10iKaxqBNL*qz_= z2vUQaJEk&|9R?{*lO93{igN8SxP!&T9;u|NNiI_F2meD}AFNrAvY(0~7eH(fsYLSk}PYK>cqDpv!VVVA|T%8v+y4voN z3rA=3UShcIX_8viwlr^|gG_=~M3Mr$;S>|O6AxW(>>p?q z`g16U6Z4Am8cUxE+<|O$4o-krY!L5&E0|ZgML_*5E`Wa#MS)i_Da{8CoyBkJAFw$$ zB6U=?-H%?XIY6?1OS&&mU~G9xbT+qOMKzzQF=$~I;IBm*DPTiIl6Q;Hq^^x+FJz#N z7Kr-!$G)WCfIIj=xkXZ4abX2jpcal(89!~=pX6cV$$!@#A|uKS(eg@%Bsn_*P6crx zpjg7jzzW2iNq?9UeiB?o{hdbtP*Zo$G4pJmYoj8yXyhW>?%ESnCPn0ZSc2zz(Q+JU zl527Wgj||X<)@J?bU{$+J;P8$bcSj?C-5M!5jGrX&2Or;vp-6MI<9Hz1oP8t=)Kh! zQ5$_+MXO$bvFwR7S0cy1UD(5kKYpIIEbH=$9^8GKmrz8b+Z&MYR{s6SxKj1|9b-~ZmR4p=r#ZmXePLPGt7iV91xX{RfXqfA)3P)2r2xn8 z`o0G&HbW6|d%%Nk@(D7n?iDsi$T)mm+S0n(&^^vz;0hw#G^G=?rqvT+%KVRLaH)h| zZ$-#~bqni*9gB2TxxQ-3n5bhoAbn`7-{F1oWlctm)huw?>WIfo0hbi$sGs@yfHW8Q zH+4-UAMY()7QUvuj@NgS7lf=@;sAHcbwv>G+dZ$-T2DhKhKl_n?%Byg6i$9Sf3Hl!BLba-xqV!7%!Yy~q{PJkUck++}g+IfW#BIaV6aVqo65$#^o0$EQP@yhPkNU@d zsB=Z691mB~bj7(UW&Q9b1BQ*Bl>idq$P9^ju!GE!B6A>x>2y+)yl1h9yQW?ewS2A? zb$xndCsE!vCZyt;#V?k8gtgUw4%iu_DN?zBuc!{kbdCia7Hz|qj4q3~-tkJ?CCY4b zE2jQKoxW)DlW_yaeMW2o@L{TixWm0IOoWbgAguEOoo2EE^LdU8+={;2faz~eFxTVr z5Zplp^y+g?BHtP|e3s;GxhnHdCW+Idhwwux&0+oPSOZSRAEV;X={e%p^IP}Ge%ELb zRT8epqlp`EYA2;>}HoQ;!uF zU{K8-S&pF0OkzF|*{RWqd*UB&iPsY9^3>=*g-d8`?YP~#-mhU8J9~I&O9;V@aV9~H zR2X1*VpSY=r_(5M+O5ogh_poR6oru+y3m*U~QNd{*OHh zMT_<|xDBwbZe*_jaJ%0F?)b~Q3~+b3-%3IthgZi|2>o8?1!omCItvW}Ip-Z9@ZA zI86Pf#IPx-OAsvr)8e=a!l`RgE!U~C%j7WRBjhmUQFI3q;s}buJA#pNq`&eqiPKi* zf7q$V%6>|wDA~7(N>IdMK(vXE&}i$;QV|mr9@G3~mp&bVhDFgYe4w`7Zz+Z!?Jrdt zFJ)@Vu5ZMm$2$25KTHJU1JX_#PDYN34qK4O^k69GhW4rl*bW^nu6HTFZ z>4|ufKw&y;sMhA;?ySTIqLuNvZpQWN3}TUGG<)_;yG!x~9u}bpcQ!?~k@a%Hq$qZ7 zF+Jqil{nMvMq_<*_v>q+#HhytUL09*hZdoC4|JhSkMX5gaMo!hh_k3m<)0B~@fQKu zcUocB)IgqAoN*MQ!UB{5_|j0DVWop|s%UpiQA-X!_7zMM8H4%-IJlZnEMsqb(lC6c zSlfZ+=XwkZ}-(~q!GSJN4clv2r{mU^-TBr;8*;}`MB#pdQi6MBUc zRNeoHgypyYov4Z$&z(^#eGEJd=R{Usv{YJ@O$T2XKvQ35@?sz}l*shLSj!S__LHFt z;D7nhRmqoP;*Ngl{6I&6jhlA0jTZqmP1wZriUrBQ1WH@w(nNM#rr7zd9&q0xm`FA~ z!N@`x^k`FpG?v(p;s{!ztPoNQob;}x^=ORi7oemqFgyi1Z_2n4A{4?8=53RB^m#dh z^duX&>e^pj+-NOJSUIJj30#AD;HFeC8+W25=&7-ak*-bxDi5moZz-`|Yoa`)sR4Ej z=lZ{-SrT$_z_(7h!)`agi5pEmmhisxTwc^fJ?-=VF9oU|E%Jeik<*B?6C&Z3eMAqSx zI?^Ei)GIgRw&!q#LjL#CJc=oLRUubLFL*G!mJUA{dk;m1m>~_!r^!KQk-$a0QVX4( z+;pzSto-<`{>OsC#)T^Q+-^BVcwfk?K z4$Wl;yK?H03$Cq1t4SWJp8n~X?gs9$dut3aSQhuPuzmAAA2ux4ura~Uz5HgTuGAMS zg{3h)GSvN1rDSe8LB^$9N3m_Ry6ZOPXPox;>S9+sRnazc$8YyVRfS(k(-%t>nuX53 zsUCm$OSB|(+=-bGe257TZs_L&`yUQ0_9crD(9%yzQjhZCq1jh%E-KPWx^#AE25%XB zDhiQcf+y1F*D7OmjJClKT~bkxjS$%xMzGOV@Ywm65E*Edp{+amJ42s-X1%_jm$<(>HztFlF(`>mU_T9bZIBT4&d=q_=9ozXMGL`skr-2vB zGG>RAv&Geo9c2v>@I>0rk)L8949RUw!NjEyY0lU9tGmjz>~m;y@Z4p;n=6t;{Lrf* zf5ah)DLfmCe(^_|L-0LF(Hnl}wv66E_OsnGfj!mn)SY`?D5cF6uaHwHYhCN%k=>416;CnJ^HLq0{>8cJu${orcMZQEL={PYvP-;(Wo7=`ESX0!OD zw9NXkO>dNufU|JwGE4-M``bFH2;I>bNR;3U6{4*(d&ZEb%U>}8!)MW z4!rrq8?kuoM9!9I z(|xA7m1gc6+>M^HUE4)r;A&FvG{{I^ZDcRY(J8ym+K>n|eRE!XfI|@F{uO zCWzXNaAU+nFY{R8gv&9ZB|2mK<)m^if8k~Og#_J>CVIIs*0IU5cKgrG?dkcoS;8Rc zGBi>5rt9Rm(+6)mR?SB#z3 zFUcj;8RajwNS`@Nrxq3T8`~;*35TDpP!tckgO%Y;9Q;oQ${s7}G$0o`xbnV`kz&>+ zdl`L}jI>m1`5#xvx7DP41&#Lf&@a-R9Kq%h3kC0uD<-q_$0CG~oW*6ydy80~4WUE- zm=Tuc&-eKEoZKggI%O2tT(`iiXp}U+!S+Q1{&MPue}p01oe*L3)TlRXim_nf-T2Y0)7 z{z$2Xm2s2A<)*_%@JQT*rWsn>Uaj>zd&HvB#7P!e9lV&)iCnbAH^xa|cN6PX6 z_GBmLsD<>>vV(L?R0S%fCPK-JfFeM$Qbo|tHx-+^-WNlEi}CrDKxZd|@eEs$W@dd< zSO4_ea+#C=>&?wmIN#Pcxb!|ps$nr=UlN^B0G^&P509Guk?%{^D~sw^O6S)NMc9XL zmfv2gg8IVhQ=dpIp zNQC&^tLSDrT|}O-&HbEEaYu0j4Y6<+Mp%=!dKS>nTTD(< zhdsOT82@{ZzK!yT;#HMJvcG`h)X%s0?)Pew@lHd?Lc2QBx9|p zY6SsGQ`Z)khqBCMh?cWDCSJf(o5|++(#bR#Q?WqZ&>v{13puFZE#8ti^Ni}C06$H> z#8OTxflVUl94zg9n0wL3buEpdb;=}juM=p=*IHeij|mQr#w#o(N+KR3O3bOE!(O}& z`=(5<$V!J@ssZK?NxwMPFFVt-eM~XbdW}p2{o8tV1OjC&gQ2jyCV0-<8J^nP z4=8PC4$W@D$=@+idTS;C?#co8&rJ`FKP4vUrByu8z7=k7L<3Z>!CU!<0q?&S0&g7! zpyl{c8}t+b_+p3azf!z4NA!QHb8%=bUX+hHWzt+m_bnHD2Tfn7$o5Ct8-0L_N=i=zuE%b54=92bzH9g%}ZsKi0+?b6)w2_N;JY|V5 zM#xhU#Mn9?Z+dpt4Q_JIT5%{abvogtW#ahJQ&c%zTPgq1OUz@8j@mnsUK=B`KF`6M zE$*JJ#={~#0mnjllEUUNPQ9R%VC&KiaH#4P$!!WsLZ><@MbMf2_bL!$ z1Y(f0>_0ulhBmHlu{>R5xo ztE^nbf?`|8(OL85{$H5Y$3`huG5(>ZhXo`osaG!rKwzh;R0~%K!He^p#hj}<&z`S7 z>RZ#St5Fd#O7tSN#YWvI2v~+*xGkH6&VJ_5SJxgG0ZPB2QO#2^T#VrRN6AL-Bx7@? z4c5XFFN)5AcDVpA=5o^=z**Bf71Jmt74`HxC^Q}&r5Ct*;putwvc}Lk$Pk;D)3det zjvW{gZKb&XD%$(dT~uzVSQ>kY`V$L9CQDuuoFK;b=kMS>eZhCdiy|IVGexCCZ>~R9 zN6(gjAH?%Zp=!4cD!o-`xEC~6pxy=~a|*P4oVj*1;;QPl{#(GN{)J_(vs#iRVp7s6 zZL@#M5GUnkqT`0{@CVsb;wIjC34?vLhlf(CGDsgNb8nHp_#4GkI%76pjyT%i7eBpy z4)fbg$F}_O=5c7Az6YKOE9|`e|b=3lE4M zS5h$vZ`4x>d3iD=HIPohP{{^yQPoIfjM^A(YqK*5kvR6@0T)h ztA<4-n@z7o#pvku>*J7qBM6wuR4(#O_vK(5ofO5_F~Yi->eU#Rr1W-IcNK)evvRg_ z&o@7Cl(6(7^B+s+#NT+TWPPP!6Ku*Ljd8?1l#YFxYOmQ?QS;?!-Ub&>63h%bu2`N4 zd55=t#ar?ixZ=UF!H_H|49@XHD821sStoeZ(s#Ll5#K!<(qed0cI)^ioiJVOCwf?f zOWk%C*B2jh^vMuXlj1f!m}Epb7yqP>@u(N7(r_4hHkzt(7d3uHbY1@B|25w`0Ep@7 z#UN{h56C172)g}AJSl(%SWXnTqj*+)X1(jOeSSW5@1L-Fm@lTCuO6-QVoGd&BaUzg_o}PcbdXP3gWIv5(>V z@%>iAQAg$DeKf7p0s?l@mN(C#7t{5_vCQSt>mcn~un0_`1D5lqru+c<*Q&E|F3yN* zL;m!i@5=fP{sN{Yp`hro`kIJ|9{1T1_u*C);oWuNb#@=>v}PJERLMLYFa;m262nj= zUu(P0eO_C+I2A>CMp$&I!U7TcUfmfRvGnIKHwh7{3a1NBLU)8wKz$AVY-%kdd0yV4 z_u;Gp7q(Sa%FhiE@xS3w@Viq_hk%FaW!MpwIKsQ)xl)}u+_zK4@tah+(zdr5G$|yQ z%i6w|18orZJW};gqO$K zLWJ7*6S1173_M*9zu%WBIa>%@HDRO62L?Q`A4w!7u$mqoJT^u&`*({!h#h`@PZ&a2 z5oxNN7kzBF%~{`L3X2q9CtDIdJeh~cqIn7nJPV_4Wx z#cH&YdB(fPJFlvY2Fityf1O!XDsIlk<0qE??Xb63gtj1zxYvQ6+~KWxx>S+ivHd%0 zor-9GRYM26eHF<2-J$ZV(`eI=L7BGl{{h87I=|zsE2z$oZ=gC%zk&Nao$^ko9N~wtx4UVIS4{rq4 z7mN=Fb^6T(T)m&pKDZQLGI3Jmgu8>x{44cMHb;djHF%~646?msIY^dC_g1&g^S28Z zrdIhI$hM7t-s0k}Otc*+%Yd1qj*Ubl)i@UzxmCH)HDWaCQ-0BOpi4-<^Q{4@k!+!R z)We&v7rYx+4*ECqPd!s&NTR6MHOP99tN-OyCFFSxuvHG85ptpItC#e1+sp4-VKhjz zlS?RGa3pjdwfi%XCejSjD0eEqC{q&d?-TV2t)l4eKuhK5K9{4a|8P1H+ll>0`j-Tb zDm}S@TTd$O`%&QrI6bf{;lk?#o97>3;~M3uNk3i?JnpZ{sp0tt$XufvBR@!otdN7; z0Rf%0J4(XJ16}2e{h63U7vo5v8P)7@HWnT4k_w%!;2|WM>oyr4Ms5=!y{DOu9$-od zSq-|3*zxNZkS)3VD|sT3C{ejGsQ~K=Dqm}YfoU=x6#2=NBiHci0QN{(xdfTu(+)5P z4(hMGfJe@62#buPCs;UmXmLKkL!9dP`3ACM1F|;E?xo}xZHc4) z6g&*G)Nwb9N=GnMWq97ec#IJJdjHWuIuZq&4p2<(v6RtGOC4xXP(Kgr-wT#<`BS<4N??4OC+X6 zCMJ4bWrJ3zG{@DcBtsx50K4*QS8|WkPA|6i(Wyk!4{bG-v1R; zF8c^iMH)q_0plSzZxYiEg-TjG@<$_uvVwekEs z98`nDKwXlIcl-jfrQ^@20@R-Ag_$iTp2_97QN?sn2g;zrIO_f#RvHIxHn#+{jB(1s&5iRIlOJtq(C%_qoFI%TE~1Ws<2`q#^YSc=#a#b2dSB<`bV{k~&ZI25$37!z|)69dKb=1eMJv;Z^tOeAA?H z>U;&y9bO`594ttc^Lzsj)xav7p{?->YmP6v5j_pK7`<=e9vyu^NdkO$Ed1Y8zRZu8 zY1to#`Nwm|RuOR$s4xSBaYgj9tPyhjzs^P z@>z(fk#4CmX7y-R*}*kS!K^;L6(l`nJSL8;KKXf{F?h*jDLjp=j{AA{vm2dYs%%Ch zZ6d4WSSo^XOE~kSD<_F8WpMNCwLX$j8ljOTzIE&vC9*`b%8V-!yCoZZ3~entoCdH0 zAUpp4?{!%TFVhO9iSI!wtYG|-&aA`eRXf{O=6j_5pZw(=);n4 zdE@PhtO{U>VQvXHz1Jiw4m%%-ZO=LSfU{OjZQFk>2`4=o3(RoInKvBCd5J|KH=9aZ zZhq*Sd-~D~VC4c~?&5v%)IE6HH%hJ?@&$)2e$B0MXz2}G9WLDiYzMeTVnz`Tx7!uB zT?2afzNL(7M@ZUy4sy>e>7f%-R5x>U>-Gd!pWV=S%w~zMMXuw3qgWnXPU$j?^iFX{ zaIkjh>9$*(W)!Lp0Nt^L%`#< zF+7ypUR>j>)bR@@qW~LY6GtXRtmUEmZ6itArFzh-N7!4&o_)imGaS+m-Q*?Yw&+nH zA=3_7tdvfV?XpxjtG@OEvPt!!G8A28u_| z|B_Sh03H(v6PRlar;sSyP(=EZ$bS;$PohrZcSf?0M4D_w62n68#GF?DNi3nRAk0f7 zh(&GhZymqIODTWLU#{Qs#Ikgng z&r7+wr^312@&Ejf|6^=-{P1aL?sni$=h{{R@h>1v;5Eu3fg)Ovz|cDYZAHdoo*eb4ie+WR_rJdav}NN>~1IM@+PY zlV`ea@d{evi95?nK}`&V!T2SjT;3;(tH;c6Kubt|?V3IcT2jiB$%gL zlpqlqanbM1&1A|A6O`056I#*?ub^-bR#8iwzvHwRdm)Wl%Q^xhL#7G1k6=njT>2@z zn`0uO?}ut404|ijbF6c>#jafh1M=G>DXk0f30;q)3)eCx<7!hwR_6O(o zn4vu{Fu(15dL)zDhK%$v%5Z4715o6~$`WvzOaEeFOTKRrFaYh2upQyd5_6@KS%4_6gSrI|U)gS7;MatTw?_ba9n1}fP~U-bxHbVY z9ly<7D%|03%(VK)H_{%njP8cx#56{C%8}K8+LQVC`Rzs4^!Qj@JVRHxMnAsZ#9;y97|EHhnD8tO*7wK$+&1UvUM)%m1%*?6-Em``~JI7ILF)&$XX4L>IbthLxx0>UwzqjH=BK>p#n94M`)Gdx0A5SD- zfup6L^zhGQ%8s12j?^yxZj@aakhTzl=QMNfP|o<|QR4wV2pM95@4jqR#uG!S|1vEi z3cW~>p<0~YlZ!*4AXsJ|uN*ksc|q20HKI)V^oFU*lcjpvL{9j&w(Y90@t{Y+JSyT) zxdyt0&~2nvX52-GI9HOHjYlm8)O(_f{wOfu#)i9Vf$yQPzK*@tOm!Ym;w0ky(?1XY z1!hEaaNr4Uy=~l#@K4|Y@tzJ7F5^=QW|wFWA`+q~{~WtH+nA^@g&S~vCR~p1*geI@ zB>B6TFt*t1@cc}nFq5L&A)}B*m+H&1{mIH~U_Q9p|GYYz<-!;fLuYGeQF?;!&Q>1E z&X6q@`UZnt10LF9W3t&LCYo;i)mL)F!8b2hp~_A~Mh!LDX& z;p`IGd_pN!tmQ4V7-+W9DlO!W$}shg`e(btrH~bVi(pofbiIc#+YJ(p9Nn&Ke?z{o zc@M9D(E=@x8eQ`8tJTtMbkAE0t;s~swcrpziwo7yC7c6BYG9yCyOxn5?aM?X|JFFr z)h+hKQ)9M3uS~Tv8V>L0244?aE6o)>lm^SF1CGrxcn6Gew_9y=M~oY7HhPEwIbwk0 zJ804Q15WxAVzbe_wWhvka^qZ6qD2o{pHWP-YBbxeZo>&1jW+6)+G31NRIU$!+TZH= zK7@PO?g$dS&m-8Ro9#Zgakv4~Pi1?PlGJI>;a^bkAi|WfIcJ6uR!@ z^9`2H;>5{fcYuVDvktf~ZxTYJ=$`{tp1B9TkBzd3f3k?UOtx`XGf;HRKpk7Cj*W!Q zqH75MBrA8+{((k0ya}hmCVH4yO@m#{?uFrc(P238IF?ZfR}K?y@y1nFmA&gQ*o3#T zz$R4Hg6&3Dsr+FRl*8Lub-*y5;_)uBs+DJ0IyZOVV3OUvl4?qK1?vQsQGprvLWUPQ5uublE!Gb9Nm zx@Ra@x#8JSXs?B;ozKqtOS3y}jMF!Sh|f)$-M)P3FV2=tc5f~%k>THkCn*tST^(TWM!amG`8q{t4sDmE{odKC0PyZBMhcn|t~MZz9cxiG?{sRM-kTTcr_z~Rhmk*kO`Z!c&37+ z+T~!u+m_=RyVn0tlVw}PxJ^fPl(Cx12qF72FLQc_n}trh!1fR>JCa2{T4oDbnv1ZN zg&S=23SG)vZfZyLA1L1!r3wn2rL`}0u6V{W`+R7P>1d&neeP`|Ew!(2^qUkzF&f<- zZ1gRNrg1$Bq0hnZ&FncZXZRFKT##&IN|%P{QboDOhuF;6Z6tWGV6=-H40xwov2ru^ zA82ISnaG|p${l6W`<5QguF(rP`lJM)j3lsJBZXB5tcUN4E)f?uw)6q_56D!zu=a}sLmzHE23bX!rfp0V_*NM{kO(9U@)Txi=sC<&Lw4T(Oug9lbW zazW0bE*6illl8aC1>xB~wdW{}#a=S3`T^8z52xs1iXT@Hs$tdNr<}ckv`PrPEB@hD z%G1@HGWZ0`S7iQ0k-6EGm7#9e5E!-;$d){9_ICeJ5D6zgvpwCeBrw9CF&f38<|ZOm zi7M^6qm{SnpBJtju9zGS_&7wXZ^M0r3td_kEL%jzpV70{e&}9kio5E@@bpu`7h1w=U?`Sn&Z)qd`l6(4bM77V!^sht&H6{78d7&~TiZdX8p| zY8P`wtMLKd!DKnGfkz;_)nuY90NH4> z`g%i?7FNkNx|2TE5L8GYX;fvjbhd%bie+b#O9Ysd5raxpg|-)zClSq~i-O0MOmvH& z(TIWdj*6fwcI(#CL!2GrtXHu-H)wuCpO8(K9pSEaQ8>Ly;qpWu@212;!#5Po?$um+ z@2I#Qm~_jnawK@>&)it;VTrd{Bg4ESY+-47t54AEj_1nQZvougPjykl^@QJO z0@tGPZUPq02YzhUC2}Js>!EVnRDGUSKmAxWX5iHvxDs(@r!}USM{7k)b z2fPixyw^{v>*iJIBB@}#^e1<8ulmBP6%|{jm9Cu%b}y^n`(v&ma>tbm#V$EOYlqM( z9!#w5(Xg6>?&#j{POs7R;l;fS{k?3Zoby!UihYRoWKFYlAf)KhqXe~w?Tsq3c$YZ5 zmj%ca98EV2Z^KGk(%ZYP)60g}KTLQ_F1}jaV*kB1RQ_c59xl?b-j-2}s6|chX{NYm zBn;pZZ68)j;yUQePeQ7D+xAcDa;Acgl3ptJ8;~&gi>ma$jdC;!N|E6$53F+aTPI@$ zU&rf(?xEKRoTFC`s030te`u%2jg3ltmctE82h&mbmS#oL2r7+HS`;$Eo=o?}=N(VC zUqPX7@>Dblu>jsWK&NrI<8t%IU)WPzE@Qwvl~8opFqAZ3pS+JG95AcppP9`g{{!3f zcz+^9cz2xgU~hy!D#9i@m=89ZQGhez2+bj_8rv9-{{^{R`xm=6wDQOo!lNZVmO>8~ z6k3{d(X+%g+^u`tr}`SW$SZMnrr86O-o*?)!Au+yRY$DHc$Rn-k)T5O*pb8vO5)qR zX}@#4x{wNOZWp@A>9}sP0FtgDj^-Qf6;o#gzAX{QIQ0G}>g>`66MTQ+?^WOq`PXv{b*Q`?!p_QPgBs7s^7GheYw=o)EyZAUlv*;0iR z$xtiZt|aGFxJTtFaGpNh6Wy>4NnWO+a>8h2C? zo-me6)w2hJcy=6}3z6J2lBgdznYvz1ce}35SWqCV6)fdW!r4_m$?)_vH#?YdlNlF} z6w5D_P+5J#k$R0gaCqCQa-6L){o6FI#5lSYFIcv#yx0gfYIYw`ze{Le;m1ARZIbX- z*=`reXzo1@IG^ph>PO*=?xB6i{O{~Ax#GpB?rPJk-1sSEJ|Ll^{}gi>O<#`q7keZj z<-wNC^5-+up5VXO?OG}hjMBhy;!f3gYN`h=_h$daZnr`&f0Y-?tFb@&slzH3yOLp3sy5Ny-f}F5XV~ucoKvjQ1hY2gEqA3%F5##uhRkkygWKUi-l&y_XI^U50L*~*2L+w@@Tg_Y;4|6&bKe`O6lyRw#iP1YR~WIR}D zLyk?5vR#8^dN$r8to4CyW@#7BvY|Vx`*vkjGS0tP-M2fd?c&aAeYmrG4T=-7I+h?P zp-k^xdP>w}xFNPHYl!W}`a=}%ofJ2oQ#){VZT%l?QeI-P<;!eaL^;T{WkORNV3%Y(s_^_pd&V45tvL@An$yxrnCU9sum z4w}v3@4VbsS-39uXx5PMm1T)8i*+{3&dByRV6H_lAz4FvuIxTnToOikFO(KjEYk>o zut~FqMqF9LiVW88_ioNn_87PJXf|r?8LZQ+;ajh)B?n@IVpYn`Vs+p4yb;&;2E>X8 zCM3J#i2cFw2J1lW&Khp@%I+wC_pT6JHO$jw->JD_?TUjT2_MPJZ_6#?S>;E|*wFVo zyEh`<<4WIfG~%(xqTnP4Z?Z+R+nB>N(MBr*Y{)2j3&SrMaUhP}Bpg@@V|=5fh6h|c z_GC_)&mD1wKn*)<9@l#nd=?HEa4i>jsZp8^tKns88&-Ps4|dD$nFpf2;fA!wR*Axu zZ}|tidegduHTY;PP*i>xE% z+V{vS^W7~R5AE=zlehJ*eSnjO)m;4#cAq@8sUMB{h^3=|*(q^*PjWkry$6plOVTP7 z`3KAP9`a-%yrMsu8SQ(v9-fC(x)`5)cyfv&4?42~)PWxa<@dm}(*;}Q)u5##{TC|@ z;9u;Kk~UZrlIV_1v+U$>BvFJ1ySW}G#0qBTud`*;K=C+;ldxL1{EL-oL!HH3WwJ6& z7%c3yX-P1(&#U3&F0fSlf3ebO{DV~!%D-5t^Z#Pm9nvX?L;}{std=+bVrB61FP0_G z$Ct%qiHaqPRTY@-tXkguiS}E_WC5Ic^ls@42!=qNbWSLGb z7wi$L1Sgd}>D9u&SZ+nHWuTIOhouH5%U$1DW8`kE=NE9}sFT=n%=Q7{I_<_XJZ$55 zg9$2s9jFwPU|CeZ1P8B*!$JwOi-u*ws3qj-R9wPxvDaA|KCBK8mZyUGWlMPQ(g#Gm z63+UQ1Bs5FBo^>iq9x~0E->2%SuNwqM~`G^39}Vkn%^AhBrHS?vuNTS5?lxNAxOdH z!~W+-Aj}Ox3hu}Yn!Q8GBAb=F1&eK(J+3IdES>U6sTFV|T=5gG7!jj9$XLYurxiM z(ovU#LynWj=Cy*^-GfC}>~)7ERfwlsU=IM-13(x0Y7}|H)pAB}gDO*%yN`kNveWcl z+0Qzqj>2;b=C!i&D-q`%B_U)@%U$fn7R@&P(RI+cFyN8+aMC;l0gv+z7|*G8Sbj!h zTV>0#afKRi)v&DW5h$V1W_guGETNI-gQ}mN{sL0K+Dt+7=jlIAf{Wb*z_lihLUXau)A9(snAGgV4fBkfPbGbM2oaVO%9Rs^!- zG7In10}Cg!1T!8K`Jq=o!q4$u&(fz^eHJ&B<4@+ylHTt;M51~M2eaho=WZp6Tb<03 z-)|mw0dXf7b$=Nn1*nl3`27cI*$$bC*vJ4IU!7MSS88~L7cOA?U zGnFkvIE-sSKaSUJeWdrXm?gvVK7EM?v2lFCp(Vr__QvX{W+jKD6b;5LF;bYD-(ts@ z2{`4L{f+9xfi{i9(H1vka`Y)tVdRtAI@^Yz60+U#S~RDYI4Su%qbw>VJw~ez?&~^G z=oou$33rDWwKR8ny779tprqqBA!+<3B;{^F+`8R@vVi17U_#Q>n~?e85;8wrK-=*C z1xdrxP-dq0FG$(mNF5f`5>%e`^68F)pptY?bWc`LOhF^PK!^bJfZRZOO5b2zEu7y~Oz#B_>*e@MpQ=pC!-cwhG9GG9h_SUQ07 zM|*05wlOtHo0w8}3#&S&yI?9lwXyBjAd^TX!wbm{uQ@UBbZwrBkkZ3fdyYn< z5x6cKAN6JbGfH%Ms>$nUcpIzTT)DSTW1e!6DrWz*cr>`p>F2Gf8r;BmiBhK|V`C~j zw=tyyHl}pI#Jti08|$a_blDn?STtn3(e07S?oJb-}iZ&_e`N!+N*?R+5R8 zhyBmk67y7qdFZ=)p2A5Hp^s|RH4*AqpwU>2v_8`2+Tnjj)!#h8qdV?U1HApVK>zSc zv*((JY%SJ0K*xK-7A^ED0hPFW#z*>H-;OP!$8T58<)g*SJM$2iEl#P;g;PSbIj;MD zb477#QFGyxKfQ3um@b@}FE5B0Sfy_qmyHtzPPmmW6wKlBVvesoBuV9O8a-v=@_z1xRbG9^uu;=`P%oBR%$4f5si@0rJ6Sn_v)6!9_;YsG!@|Y_vU(o9Kap zAG(={@ChaUAbZB1){V>5j%wdinHdhYqvP1hL%Ey*T4hzoDEgmqZ)k%rP?dcmK9lUgv_N8m?>^)Q>LOwX~!u_>} z)uQ9>KEDHlX_Yv~>vyUhksDVJ*W;+g_L83e@>XxMtFf;=KG7YYxSe=>J+8+DOdz|* z%$@lB#tRp@GLjgPrAc5qez`J}b~F$Bh{vAY$oJnQ^SBu6N?;&xJ+JL}s`xk~%Jkd) zrRjdF}TJ# z&xMnT%Y&PIOr$K`S!)sg(_P?^YTWN~*qeJ9R11{XQkl$>jLxy-n&f}~m0 zV4ENp;C8ZWK-$|8&p(iDZq6wZ>!tznFWfp!5j8mX{;=PY2|@!iac_IUaW2S_ORpIV zuI~2Sn9ox8<6UdI2^`=V(-IiM022qrHp#(d&bcd>kM@q`5J@oPmy4WB`~zW|sds?5 za2O*FSEil zG94lu9Pi1wNs<|t9?!YNy&&mejW2inu3QFrl&UbeZFSQP;{ApJ@*W4~KBy$S&s{r^ zMx4yLq}#n&98bY;(|@4y%>WAr_e@eJhzSCq4;)loM)?w23>`;Inn#C`p5*mT=;b7$U6p70(7dfZ8Os@rKZuKP3hj4w3 z@317R=;-!c6&3|9C(i9SX}Siiya+73D=cD|^D&Ae4v-6nZIz@~IC##BzE|akIhtUjg!YtA=ToUV1BrKwiP5g%BCzC3?azO$#5)_`sh-=If2EjEZPN0f%Ztf ztAi{Dnb?oh0U`1mJi;(v+ClCq_jzZ6fIGiP=&NQ)(O1)`O`JXw={dQl0u;>~J;I`e z?k1(m$!)R7Kfv;*2cMn8ZDw8)m!}!Txp@%q=nd{NH#Vmx9Ub4e9UWusKrLdQrr-@& zW@J1`fU@6E@h(7jDZh`EYBMqgZrt_*eu^DVl9B)`C)xHPj;wJ)LQij_VlEO5J!e;s zTAd2gP{_5D{y9jtLdNx8YBjpPB*cD9V1;#gra`3@BMRucn({=1 zF1T)=zXA2kDpkwFu`4o+T0Q;-^C$VCa*3#t*L^vR=b<-w3+H$Ij;tzAkAT z5+gI~^7OJ8QU!i~lBb71914>jAF_nkkj#p^W8HR8w zTIlTtycbxx9JYBTin{S!Z;p;iZ{v};qhapB@vIimQR5%^RYwiatqhPZM1|Lh?w?Zd;f1W_}zxhLTa0#uBOW@3J2+pyGf&1l=D%h-BL)l_}wG*S-e@ zT0{R&SH{lxc$6>jzp(dd11b-6H&HzDzX~Z>H`@}~bkojNb{OA+dnUb)g>Ta}k?EOE z(5kh$2kZuJXMkx2VNc0aY%5}r14r4>*DJD2bwNS0`no)02(v*ZkP2D?w2Eu)&%A*A z;0uMiK~ia=9^l8qe|zh68#}TEq3;Ve4(=GLy^EEClX8@7F0pxzrrqY-AG)Ic0IVE5 zwp%ei*t8&!@8~o>TjY1?S8xPAP=BC~Io9{r8x>*2@;o|ET7>NYdP^?tsFKylT!&x<2lWHouh<<4NA4X(>;m~Nq>|F{+u|6^ z1FaL?y3UGYz7|Q1;)3rWt`q!zXQ0rJP44pMk62 zbhwV}mC~r<06nk92a?T3e%uQwRMsZo9kd+>SeZpvkRL#fbB5CJA%vy_ZCvj3W0pnJr|QFcm2 zkE3=l&}3c8o_UXCAc}9hkEze|O!6oO$xf2Vd^Z4a+5poF z)}u{+OAapq0j(1~MpB(SU%@@(^QfPmMk2H_yL390)CTz>HI$271dU=1nF~i`wMl@M@`q26cp6w1F*+z2)WMt%BQVWwS$o5fx-qKptKciDMB+jzn z42FqxdwM~@ccZJJaoPLh6d5XlD2hOKV1FbX(ZLClgeu-I9jNC6&tfSoSm!}rWedIf zj2sg}SHhq_(}m_x-N2$*d0v$A%_W86A#;bw@*E#oPk#q%qLLqk z2Hq&ktDcF~U5)CCMlMfv^X3Zfg+ZO;p4QJFooC*jejZ<%R;!J7id}ntQ`Bo}FIk_X zPrLhc;RBM&{uTauD8Tayl&}&1uC*%O>lrK?daoAlmsi(8h9%YOgoM77w9{ic2^fr10Sx`+gI2y$hG+^8Fc*d#;;tFmD zIfdV_A3RWhpgVI1PgBt=?LQ4i3D^p2&Ldvy_!%uv@!l()xq!Uz6QYVp}Z9!C?0(&5}d7eZvkH*7a%J zauS@9u|%cv0Dzd zy)R&9CgLL(3rAdWz*u3?E0ZO94sW?pGTr!O5(UBKfX}_~NJ387NFsP8?t4*(h8FD9 zKbm?6oF5MB(koX~T`at#TaA~>qvV0f>XEyl`r7dwHTrc!jegxwb*<%&staa!RM*+L zqF;rT^Z~;^P<6HRj((MZQIzG5T3UERzjsS5J^z7z9W5i>>5l5#^mp`ow?1o@QNiRN z=vUi1Qi6UQT|j&A;<(49mZ#>%(c_Y~8)%zoIa;;U#YWwcK|WCIycnoIZe_k`w$W&y z_6PVQVS1pbNT^VrP_kQ1*0&aBh77g7wfyQk&sS8HB?DS*)bIm0)UbjJ%5q#6O5u-z z9%nT#dTG`8028fj2v!W3i!}sR)sj2PL)OVwYYeKs?e`D#yuwT@-upAzy~bYhD&B6F zrJpRFrH2nKb|2F3c1@+NJm+_#);RF9ala&1a6VwM`x91xT}?6YV{gB@Mo-;X>~S5a z(ktDd`%4b) zFFE+X%+ZJ4PIW@GoJNc>G^$8V!<=veJaI4Mi+z{RZ}_2O{OAh#8;V zh-8?UB9=_215T1qR;UPNhKf+$L=i;}U6X`VFij|lNy06O&1o!1fhcR##1h%>yUQ|m z0yv@!QW44`6|rPCuN6f}D5F)xlInbGLH$YVZTuvz?ij)D$8W!9b)Y7>G0%q2XMNCTXLM8+RF5i(QJP;LO9e4U?FES|=MOf77g43qj!zmRNnUwOxCUuju7VzT}S(% zK1t%P(Fb+2?UTCM_DOw`*WEfeb=T;F`ZU^VEo@VtQKiEXt{M2+{>LrnJJO zl*CL*Nz9~HCubr7T_9ALQVWw(YGG1JViu*%$fQ^&-J+bNl&dhQy@r_uf^hd&VM;Ab zN(s!Ow9nU+XMQ^1syqt0;gvWyaxNLS#6NeH@xqwK)H2_qw#LrQa=N*#Ftv$dQrj+H zEIt;tH$vWZb`06ThDO+Jo5{lp8?ZDPZirAosHKCsQ&gIhZyj2h*mc1G1@Yzmv+E+KzEi#h_Ln=Xk>| zD@uZJ%5#CCt?Ug-shCNrAu=iH7Y)j@D0lWT2ZP$%NwcwHVQPp>N)3@oDN&fzwzi2i z3u{vPEo(}x6>ZA%@g}Molo}(GT4VI?&Stu*FR3Pe5f}RurCg#-$z81c7~jQqmM5ij z+Hi;l6|{zB+MJSXW=wm}pX3 zzf9`WII1wwras9_c|)7p``A=(7)5WW9s*4en^SS3O=+!AlyMkuQ>>Ss5^xFgKWA|` zH80!L4q$&CdXo$fxq>=M6gK{o&r3x?8A@$YftxoKDN2Llr3uxvY*V_A7PY>99$BLp zfuz)`#-{WTS=65X-jcPMZWy8d5XY7%-jye^oHeyX@y0|sMuS?GJ7vB_>Xg)^_F_HFf))YkZ32_FYg2N- zZE8n#uZ>WmnZ&ji_V25plw(#7EHgJ8-&g!9ZU9t??G%JS)sP-MFpL^;A;<*~KZK^@ zL-;z}D0-^VuoSaREy1zTdld2<}MWN=HjuS)Y#jZhho zjd`XG@5|v1H6Bog`2?8@@Z(MjaTNvCIJbIS*MYU8yC#C$%kTP@dGfd-mQdyCD7sgf z!O;`z9woj~lpGu2h&L8quA;uh;>r<*xZPZn7|U9EF=)XfvDJ-C6F&WHqq_gq@seqK z2B7jbjW%o_s#D&c16)I#=w2*=qar*nfLe~||9u1)m>1ag@D1-yIx3Hg&9aQdc~*3( z7(2HYZYM5`(nExzjAX`RX$PA5N9G0c$FJO(S%H~-yvPvyh@-q>Xq@Ds%;-0`@+eLX zG;vOGdj*S5ljh(l0gy2s1^V>Pw#{X3E4#tK-W+l=x#OV?{7sgllH#_4@vse%U5Z4O zqfNETZF71H*y1Xm2>d9CSAKqRIlZf9K-#fGxnss3JL9 z!-MoCfS`9s1DIY2F1&I68CC|zg;VEP7tS6`&Dn#gIenOAbNV>T=JbJ<$?2?*3-__V z`UL#KsWb2k_fdYv(VWeF)B}CWec?XJ#W?P>xsQ6FPrEPNM?KIk-{8z#j|=yuzpTRD zpWGKcFf&7JPM>ODxG#EO=7(Ikk8=5BAFQg~pWGKcFf&Ci+(-F+vLjZ`?oaNc9(=MR zGi&6+eUhL1Xdmv|Jac4^{K0+dhx;}!m{}heZuMiOYT%^-%^Blsa>n?YoIDTmD8*}V zS}r!H!Jm6kv+V;P{*>kFqubK#CVVDwsKPJV#xD0inu+2(YBa^bXOY;K=SOf|$Y zjRCGfj?+>~X#am>=aOVQszcEUof~2PxL>2F`48l=Wdj>Yez!W=$rU~zBq4s12O87+ z;7pHwZm)Eo5;-7IEd|wm&b(pUS=ZUb8?&|rqtUNB7Y|(O(BCg6lwol|tq)(^5)Kj- zOCBPrHYGJkVx}~Fj&J|smf=gXE-9mgeEdQ-CXILh;tu5*qq0FxX31{*DTswqHot@n zbJ(p~`HiD{(dCQN`t!vd;=xwp0vjb&a$NTpr|0l5PG4|+xZ7|>Cq6jsK3y)U7$a0B zdpk(&#lE=D7Ao!-k`M}!clQ#<*)J+?$0PTA`-W5k*>a{l0qVSaGNFh4j)F5PfW-i9yE zc;FAt>95O?U!7pG1I)Ku%H*Hp1@@UO^0TDSR8~O9Qbr0OPJEo()@9cr?7(ZkR&DZN{B4tD8 z+X*r(LBK~PBF)>4Z0a?Ee8u8%`==#CjF9Q+4X0~u*!t&jiJS|Kz#N^k)O2|x}FOvQ=6gteK zrO#r7QiSt?sHBoXot%O8rva66#PeCF^a*7|Gb($5Z1Vt~Q;Plbuaj48-G3hcow0@n z>IAM2p#2T#e*^Bn0I$~mA0Yk>NE3KnY}P8EQzh8RY@V?;*Z3aXiNmv5ahQ! ziKvKbzOh6mlRql;qE6y|f4MJs@9sEA;U7Z9J-IGg8CklR=56d>Qiyh84$}8m9;6y% z$@ZOP`lXR=pR;%uDzZfZkCW&%T^@M0rkTh>dsEbzzt~8cRlF~P-5Z4;L*&e0X8QIZ z(%cFmwf7Bkk@Df2+wB*5!SZpbA5Zk1L}S|!8s$bwV}*#yObGjkBZ0S4+=Lr4Ha}BTXU$a2U0HhYO3mQ_25uO z65VT;k%UKW$NMf%%WY(TQ;;*;br*-y4%(;8XuDKnAArI*IF%k)x(Z``H$TXW$qOZ| z2UVHU!|hT#aFH5t&x2@CGlexL8-pm`_(mQb0?UkLmKYiquZQC12OYHe)M4J(Y?xTO z7}NIcQ-@l#-LiZI!bq#g-4|$ zpv8NXiR}^Jw@~fJPO7q!L3{Gk$vrvmjAQ&QkJ1+IQ@65RDfz9b?a=4D&|!0yG6U&S z`7jYYxUCNs3u8w4UFvX&(0!~02DP<%UnS*lIn;LO>%DDrs5OvN^ILYPC9rvKMk){( z%t*|owpK`OJ*-1e!u^+1WX!BHTxzKWLRWkXNpli5t%%xM{m%0Z;o?aJa_Mt>PHLAM zZ1wV;MNpf`QaY|{Qh`@fS}lA^hde%Yv$Y9jL=njG&)mmYvKlHV$J2`Qi^h(cln&OE z)(W56s$b{ZU3sCbu*MdWrOx_8b*8kl@F`k@K6)tiBz`$xM6n$`?Lvv-swvY+m&#;E z&&N=#v1zPqb$wm|U#QuqDXkSgrOn8tXx%#BCnJNxePEGXHU1@4i*v#Xc)hXXoMZHx zE~VX=OYshi1U*8Hq0E&jJsG)ikA} z6ra-i>Qg+q1=)iclSqR09BIk47xSqB(*rF0zO`C*8F0mCsF$LQR(RBo|bl;;Wuzr2dJwseYTPop1qrU4kH{lMx)Q@;q>{XZ?=#d5bfNRzpg~52 zB->Fv2B!2D0iNkwDjO<0lr}n_qOI8QPz@_gMVZ4Ck2*KIQ`=bSjG1#4m#S>RpmK01 zJ$w0-X}(9<^B$KnLo|%nQf7#Ty;-UlDr+h!4HK$5lreIb(mU_>aGRF0@^YwixIZa# z%;r*7_8w(a+NF$ryOh-&k21%MF7=<}<>;43=?zm<>9*9lssvU&%m~b-^hVoVjY=n) zcxS_^qpZm&g-aQwuv8|+eNv<`2eF(CEFc> zaE0t(I#da`8)Y%A{sZd{mJJp!2h(vgiiA7Zy;W#p#bSx}L?zfASi^pQr{oyLPT~XW z4z~2R&!Xh1qGFB0jy*D|hdLr>U_CN+xU)_eC1v3A=ro5OlU$Q1l&~{=mLkt15**we z_*c*PQ&c3k2g{6^gGe7+gSrYs4r{lV_S_%X;kpB3M^|s7zV))$)^bTk9gz$wga@Q7 zPsfondLi61u6J(C?KQ>g9s*94hCM58jlW#X!cgFm4Y{kU_U;(SK9+@&(p#%{x*d8*U?&9 z=k#Z77?GN#Ww@k^$)!Ozj?N_;RvO_Ra${YI+i4=EFK){ozlbJE#vDw2NYZrV3fp}j z0}3|FTfciOSH|M8a-U56^u>{whTD=mkP)LaWHf$;%&d}`)%+af*P9Re1-8#-a!=GR6_Qr_>AAObW8? Date: Tue, 20 May 2025 17:37:31 +0300 Subject: [PATCH 02/23] Added db to sql when creating or dropping table. --- .../flink/connector/clickhouse/sink/ClickHouseSinkTests.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java index 2e375ed..cad95ed 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java @@ -47,10 +47,10 @@ private int executeJob(StreamExecutionEnvironment env, String tableName) throws void CSVDataTest() throws Exception { //String path = ClickHouseSinkTests.class.getClassLoader().getResource(".").toString(); String tableName = "csv_covid"; - String dropTable = "drop table if exists " + tableName; + String dropTable = String.format("DROP TABLE IF EXISTS `%s`.`%s`", getDatabase(), tableName); ClickHouseServerForTests.executeSql(dropTable); // create table - String tableSql = "CREATE TABLE " + tableName + " (" + + String tableSql = "CREATE TABLE `" + getDatabase() + "`.`" + tableName + "` (" + "date Date," + "location_key LowCardinality(String)," + "new_confirmed Int32," + From ab73284be4dad90a85e831b598dc55dd508af1a9 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Wed, 21 May 2025 14:27:03 +0300 Subject: [PATCH 03/23] Clean up code & add handleFailedRequest method --- .../sink/ClickHouseAsyncWriter.java | 22 ++++++++++++++----- .../clickhouse/sink/ClickHouseSinkTests.java | 2 +- 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java index 7d9c6f4..dfb7517 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java @@ -80,12 +80,10 @@ protected void submitRequestEntries(List requestEntries, Resu } LOG.info("Data that will be send to ClickHouse in bytes {} and the amount of records {}.", totalSizeSend.get(), requestEntries.size()); out.close(); - - // .setOption(ClientConfigProperties.ASYNC_OPERATIONS.getKey(), "true").serverSetting(ServerSettings.WAIT_END_OF_QUERY, "0") }, format, new InsertSettings().setOption(ClientConfigProperties.ASYNC_OPERATIONS.getKey(), "true")); response.whenComplete((insertResponse, throwable) -> { if (throwable != null) { - System.out.println(throwable.getMessage()); + handleFailedRequest(requestEntries, resultHandler, throwable); } else { handleSuccessfulRequest(resultHandler, insertResponse); } @@ -98,8 +96,22 @@ protected void submitRequestEntries(List requestEntries, Resu private void handleSuccessfulRequest( ResultHandler resultHandler, InsertResponse response) { + LOG.info("Successfully completed submitting request. Response [written rows {}, time took to insert {}, written bytes {}, query_id {}]", + response.getWrittenRows(), + response.getServerTime(), + response.getWrittenBytes(), + response.getQueryId() + ); resultHandler.complete(); - LOG.info("Successfully completed." + response.getWrittenRows()); - LOG.info("Successfully completed." + response.getServerTime()); } + + private void handleFailedRequest( + List requestEntries, + ResultHandler resultHandler, + Throwable error) { + // TODO extract from error if we can retry + error.printStackTrace(); + + } + } diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java index cad95ed..f5bacfe 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java @@ -82,7 +82,7 @@ void CSVDataTest() throws Exception { clickHouseClientConfig ); // in case of just want to forward our data use the appropriate ClickHouse format - csvSink.setClickHouseFormat(ClickHouseFormat.CSV); + csvSink.setClickHouseFormat(ClickHouseFormat.TSV); Path filePath = new Path("./src/test/resources/epidemiology_top_10000.csv.gz"); From 9d641d9ae8d27e5689f7901c2fa7ce9c78c93ac5 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Wed, 21 May 2025 18:42:17 +0300 Subject: [PATCH 04/23] Adding scala tests --- .github/workflows/tests.yaml | 11 +- .../build.gradle.kts | 58 +++++++++- .../clickhouse/sink/ClickHouseAsyncSink.java | 2 +- .../clickhouse/sink/ClickHouseSinkTests.java | 2 +- .../connector/test/FlinkClusterTests.java | 13 ++- .../test/scala/ClickHouseSinkTests.scala | 105 ++++++++++++++++++ 6 files changed, 180 insertions(+), 11 deletions(-) create mode 100644 flink-connector-clickhouse-base/src/test/scala/org/apache/flink/connector/clickhouse/test/scala/ClickHouseSinkTests.scala diff --git a/.github/workflows/tests.yaml b/.github/workflows/tests.yaml index c1aef24..aa17e91 100644 --- a/.github/workflows/tests.yaml +++ b/.github/workflows/tests.yaml @@ -37,4 +37,13 @@ jobs: CLICKHOUSE_CLOUD_HOST: ${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_HOST_SMT }} CLICKHOUSE_CLOUD_PASSWORD: ${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_PASSWORD_SMT }} with: - arguments: test \ No newline at end of file + arguments: test + - name: Setup and execute Gradle 'runScalaTests' task + if: env.SKIP_STEP != 'true' + uses: gradle/gradle-build-action@v2 + env: + CLICKHOUSE_VERSION: ${{ matrix.clickhouse }} + CLICKHOUSE_CLOUD_HOST: ${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_HOST_SMT }} + CLICKHOUSE_CLOUD_PASSWORD: ${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_PASSWORD_SMT }} + with: + arguments: runScalaTests \ No newline at end of file diff --git a/flink-connector-clickhouse-base/build.gradle.kts b/flink-connector-clickhouse-base/build.gradle.kts index f4044f5..5f16cdc 100644 --- a/flink-connector-clickhouse-base/build.gradle.kts +++ b/flink-connector-clickhouse-base/build.gradle.kts @@ -4,10 +4,12 @@ */ plugins { - kotlin("jvm") version "1.9.0" + scala java } +val scalaVersion = "2.13.12" + repositories { // Use Maven Central for resolving dependencies. mavenCentral() @@ -20,7 +22,6 @@ extra.apply { set("testContainersVersion", "1.21.0") } - dependencies { // Use JUnit Jupiter for testing. testImplementation(libs.junit.jupiter) @@ -29,6 +30,8 @@ dependencies { // This dependency is used by the application. implementation(libs.guava) + implementation("org.scala-lang:scala-library:$scalaVersion") + implementation("org.scala-lang:scala-compiler:$scalaVersion") // logger implementation("org.apache.logging.log4j:log4j-slf4j-impl:${project.extra["log4jVersion"]}") implementation("org.apache.logging.log4j:log4j-api:${project.extra["log4jVersion"]}") @@ -60,6 +63,28 @@ dependencies { // testImplementation("org.testcontainers:testcontainers:${project.extra["testContainersVersion"]}") testImplementation("org.testcontainers:clickhouse:${project.extra["testContainersVersion"]}") + testImplementation("org.scalatest:scalatest_2.13:3.2.19") + testRuntimeOnly("org.scalatestplus:junit-4-13_2.13:3.2.18.0") +// testRuntimeOnly("org.pegdown:pegdown:1.6.0") // sometimes required by ScalaTest +} + +sourceSets { + main { + scala { + srcDirs("src/main/scala") + } + java { + srcDirs("src/main/java") + } + } + test { + scala { + srcDirs("src/test/scala") + } + java { + srcDirs("src/test/java") + } + } } // Apply a specific Java toolchain to ease working on different environments. @@ -69,7 +94,36 @@ java { } } +tasks.test { + useJUnitPlatform() + + include("**/*Test.class", "**/*Tests.class", "**/*Spec.class") + testLogging { + events("passed", "failed", "skipped") + //showStandardStreams = true - , "standardOut", "standardError" + } +} + +tasks.withType { + scalaCompileOptions.apply { + encoding = "UTF-8" + isDeprecation = true + additionalParameters = listOf("-feature", "-unchecked") + } +} + tasks.named("test") { // Use JUnit Platform for unit tests. useJUnitPlatform() } + +tasks.register("runScalaTests") { + group = "verification" + mainClass.set("org.scalatest.tools.Runner") + classpath = sourceSets["test"].runtimeClasspath + args = listOf( + "-R", "build/classes/scala/test", + "-oD", // show durations + "-s", "org.apache.flink.connector.clickhouse.test.scala.ClickHouseSinkTests" + ) +} diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSink.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSink.java index 20caa44..54f30bc 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSink.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSink.java @@ -23,7 +23,7 @@ public class ClickHouseAsyncSink extends AsyncSinkBase converter, int maxBatchSize, int maxInFlightRequests, diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java index f5bacfe..cad95ed 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java @@ -82,7 +82,7 @@ void CSVDataTest() throws Exception { clickHouseClientConfig ); // in case of just want to forward our data use the appropriate ClickHouse format - csvSink.setClickHouseFormat(ClickHouseFormat.TSV); + csvSink.setClickHouseFormat(ClickHouseFormat.CSV); Path filePath = new Path("./src/test/resources/epidemiology_top_10000.csv.gz"); diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/FlinkClusterTests.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/FlinkClusterTests.java index ca3bd36..9ee6463 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/FlinkClusterTests.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/FlinkClusterTests.java @@ -6,31 +6,32 @@ import org.junit.jupiter.api.BeforeAll; public class FlinkClusterTests { + @BeforeAll - static void setUp() throws Exception { + public static void setUp() throws Exception { EmbeddedFlinkClusterForTests.setUp(); ClickHouseServerForTests.setUp(); } @AfterAll - static void tearDown() { + public static void tearDown() { EmbeddedFlinkClusterForTests.tearDown(); ClickHouseServerForTests.tearDown(); } - public String getServerURL() { + public static String getServerURL() { return ClickHouseServerForTests.getURL(); } - public String getUsername() { + public static String getUsername() { return ClickHouseServerForTests.getUsername(); } - public String getPassword() { + public static String getPassword() { return ClickHouseServerForTests.getPassword(); } - public String getDatabase() { + public static String getDatabase() { return ClickHouseServerForTests.getDatabase(); } } diff --git a/flink-connector-clickhouse-base/src/test/scala/org/apache/flink/connector/clickhouse/test/scala/ClickHouseSinkTests.scala b/flink-connector-clickhouse-base/src/test/scala/org/apache/flink/connector/clickhouse/test/scala/ClickHouseSinkTests.scala new file mode 100644 index 0000000..578e8dd --- /dev/null +++ b/flink-connector-clickhouse-base/src/test/scala/org/apache/flink/connector/clickhouse/test/scala/ClickHouseSinkTests.scala @@ -0,0 +1,105 @@ +package org.apache.flink.connector.clickhouse.test.scala + +import com.clickhouse.data.ClickHouseFormat +import org.apache.flink.api.common.eventtime.WatermarkStrategy +import org.apache.flink.connector.clickhouse.convertor.ClickHouseConvertor +import org.apache.flink.connector.clickhouse.sink.{ClickHouseAsyncSink, ClickHouseClientConfig} +import org.apache.flink.connector.file.src.FileSource +import org.apache.flink.connector.file.src.reader.TextLineInputFormat +import org.apache.flink.connector.test.FlinkClusterTests +import org.apache.flink.connector.test.embedded.clickhouse.ClickHouseServerForTests +import org.apache.flink.connector.test.embedded.flink.EmbeddedFlinkClusterForTests +import org.apache.flink.core.execution.JobClient +import org.apache.flink.core.fs.Path +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuite + +class ClickHouseSinkTests extends AnyFunSuite with BeforeAndAfterAll { + + val EXPECTED_ROWS = 10000 + + override def beforeAll(): Unit = { + FlinkClusterTests.setUp() + } + + override def afterAll(): Unit = { + FlinkClusterTests.tearDown() + } + + @throws[Exception] + private def executeJob(env: StreamExecutionEnvironment, tableName: String) = { + val jobClient : JobClient = env.executeAsync("Read GZipped CSV with FileSource") + var rows : Integer = 0 + var iterations : Integer = 0 + var continue : Boolean = true + while (iterations < 10 && continue) { + Thread.sleep(1000) + iterations += 1 + rows = ClickHouseServerForTests.countRows(tableName) + System.out.println("Rows: " + rows) + if (rows == EXPECTED_ROWS) continue = false //todo: break is not supported + } + // cancel job + jobClient.cancel + rows + } + + test("csv data") { + val tableName = "csv_scala_covid" + val dropTable = String.format("DROP TABLE IF EXISTS `%s`.`%s`", FlinkClusterTests.getDatabase, tableName) + ClickHouseServerForTests.executeSql(dropTable) + // create table + val tableSql = "CREATE TABLE `" + FlinkClusterTests.getDatabase + "`.`" + tableName + "` (" + + "date Date," + + "location_key LowCardinality(String)," + + "new_confirmed Int32," + + "new_deceased Int32," + + "new_recovered Int32," + + "new_tested Int32," + + "cumulative_confirmed Int32," + + "cumulative_deceased Int32," + + "cumulative_recovered Int32," + + "cumulative_tested Int32" + + ") " + + "ENGINE = MergeTree " + + "ORDER BY (location_key, date); " + + ClickHouseServerForTests.executeSql(tableSql); + val env = EmbeddedFlinkClusterForTests.getMiniCluster.getTestStreamEnvironment + + env.setParallelism(1); + + val clickHouseClientConfig = new ClickHouseClientConfig(FlinkClusterTests.getServerURL, FlinkClusterTests.getUsername, FlinkClusterTests.getPassword, FlinkClusterTests.getDatabase, tableName); + val convertorString: ClickHouseConvertor[String] = new ClickHouseConvertor(classOf[String]); + // create sink + val csvSink = new ClickHouseAsyncSink[String]( + convertorString, + 5000, + 2, + 20000, + 1024 * 1024, + 5 * 1000, + 1000, + clickHouseClientConfig + ); + // in case of just want to forward our data use the appropriate ClickHouse format + csvSink.setClickHouseFormat(ClickHouseFormat.CSV); + + val filePath = new Path("./src/test/resources/epidemiology_top_10000.csv.gz"); + + val source = FileSource + .forRecordStreamFormat(new TextLineInputFormat(), filePath) + .build(); + + val lines = env.fromSource( + source, + WatermarkStrategy.noWatermarks(), + "GzipCsvSource" + ); + lines.sinkTo(csvSink); + val rows = executeJob(env, tableName); + assert(EXPECTED_ROWS == rows); + } + +} From c160c685f018ed2986e783e9ee4a7d9191940f61 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Wed, 21 May 2025 19:18:43 +0300 Subject: [PATCH 05/23] Split tests to workflow for Java & Scala --- .github/workflows/tests-scala.yaml | 40 ++++++++++++++++++++++++++++++ .github/workflows/tests.yaml | 13 ++-------- 2 files changed, 42 insertions(+), 11 deletions(-) create mode 100644 .github/workflows/tests-scala.yaml diff --git a/.github/workflows/tests-scala.yaml b/.github/workflows/tests-scala.yaml new file mode 100644 index 0000000..4005630 --- /dev/null +++ b/.github/workflows/tests-scala.yaml @@ -0,0 +1,40 @@ +name: Apache Flink ClickHouse Connector Tests CI (Scala) + +on: [push] + +jobs: + build: + runs-on: ubuntu-latest + strategy: + fail-fast: false + matrix: + clickhouse: [ "23.7", "24.3", "latest", "cloud" ] + name: Apache Flink ClickHouse Connector tests with ClickHouse ${{ matrix.clickhouse }} + steps: + - name: Check for Cloud Credentials + id: check-cloud-credentials + run: | + if [[ "${{ matrix.clickhouse }}" == "cloud" && (-z "${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_HOST_SMT }}" || -z "${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_PASSWORD_SMT }}") ]]; then + echo "SKIP_STEP=true" >> $GITHUB_ENV + else + echo "SKIP_STEP=false" >> $GITHUB_ENV + fi + shell: bash + - uses: actions/checkout@v3 + if: env.SKIP_STEP != 'true' + - name: Set up JDK 17 + if: env.SKIP_STEP != 'true' + uses: actions/setup-java@v3 + with: + java-version: '21' + distribution: 'adopt' + architecture: x64 + - name: Setup and execute Gradle 'runScalaTests' task + if: env.SKIP_STEP != 'true' + uses: gradle/gradle-build-action@v2 + env: + CLICKHOUSE_VERSION: ${{ matrix.clickhouse }} + CLICKHOUSE_CLOUD_HOST: ${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_HOST_SMT }} + CLICKHOUSE_CLOUD_PASSWORD: ${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_PASSWORD_SMT }} + with: + arguments: runScalaTests \ No newline at end of file diff --git a/.github/workflows/tests.yaml b/.github/workflows/tests.yaml index aa17e91..4ff495e 100644 --- a/.github/workflows/tests.yaml +++ b/.github/workflows/tests.yaml @@ -1,4 +1,4 @@ -name: Apache Flink ClickHouse Connector Tests CI +name: Apache Flink ClickHouse Connector Tests CI (Java) on: [push] @@ -37,13 +37,4 @@ jobs: CLICKHOUSE_CLOUD_HOST: ${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_HOST_SMT }} CLICKHOUSE_CLOUD_PASSWORD: ${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_PASSWORD_SMT }} with: - arguments: test - - name: Setup and execute Gradle 'runScalaTests' task - if: env.SKIP_STEP != 'true' - uses: gradle/gradle-build-action@v2 - env: - CLICKHOUSE_VERSION: ${{ matrix.clickhouse }} - CLICKHOUSE_CLOUD_HOST: ${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_HOST_SMT }} - CLICKHOUSE_CLOUD_PASSWORD: ${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_PASSWORD_SMT }} - with: - arguments: runScalaTests \ No newline at end of file + arguments: test \ No newline at end of file From 97a0fe4ef5f0a71ca3f03f6ec2fbb8e6ef1c8702 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Mon, 2 Jun 2025 12:10:34 +0300 Subject: [PATCH 06/23] Adding POJO support --- .../build.gradle.kts | 11 +- .../java/com/clickhouse/utils/Serialize.java | 183 ++++++++++++++++++ .../convertor/ClickHouseConvertor.java | 20 +- .../clickhouse/convertor/POJOConvertor.java | 16 ++ .../clickhouse/data/ClickHousePayload.java | 1 - .../sink/ClickHouseClientConfig.java | 18 +- .../clickhouse/sink/ClickHouseSinkTests.java | 157 ++++++++++++++- .../sink/convertor/CovidPOJOConvertor.java | 25 +++ .../sink/convertor/SimplePOJOConvertor.java | 32 +++ .../clickhouse/sink/pojo/CovidPOJO.java | 101 ++++++++++ .../clickhouse/sink/pojo/SimplePOJO.java | 90 +++++++++ .../connector/test/DummyFlinkClusterTest.java | 3 +- .../clickhouse/ClickHouseServerForTests.java | 12 +- .../flink/EmbeddedFlinkClusterForTests.java | 2 - 14 files changed, 644 insertions(+), 27 deletions(-) create mode 100644 flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java create mode 100644 flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/POJOConvertor.java create mode 100644 flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/convertor/CovidPOJOConvertor.java create mode 100644 flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/convertor/SimplePOJOConvertor.java create mode 100644 flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/pojo/CovidPOJO.java create mode 100644 flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/pojo/SimplePOJO.java diff --git a/flink-connector-clickhouse-base/build.gradle.kts b/flink-connector-clickhouse-base/build.gradle.kts index 5f16cdc..8aa5870 100644 --- a/flink-connector-clickhouse-base/build.gradle.kts +++ b/flink-connector-clickhouse-base/build.gradle.kts @@ -12,14 +12,16 @@ val scalaVersion = "2.13.12" repositories { // Use Maven Central for resolving dependencies. + mavenLocal() mavenCentral() } extra.apply { - set("clickHouseDriverVersion", "0.8.5") + set("clickHouseDriverVersion", "0.8.6-SNAPSHOT") set("flinkVersion", "2.0.0") set("log4jVersion","2.17.2") set("testContainersVersion", "1.21.0") + set("byteBuddyVersion", "1.17.5") } dependencies { @@ -28,6 +30,8 @@ dependencies { testRuntimeOnly("org.junit.platform:junit-platform-launcher") + implementation("net.bytebuddy:byte-buddy:${project.extra["byteBuddyVersion"]}") + implementation("net.bytebuddy:byte-buddy-agent:${project.extra["byteBuddyVersion"]}") // This dependency is used by the application. implementation(libs.guava) implementation("org.scala-lang:scala-library:$scalaVersion") @@ -39,10 +43,7 @@ dependencies { implementation("org.apache.logging.log4j:log4j-core:${project.extra["log4jVersion"]}") // ClickHouse Client Libraries - implementation("com.clickhouse:clickhouse-client:${project.extra["clickHouseDriverVersion"]}") - implementation("com.clickhouse:clickhouse-http-client:${project.extra["clickHouseDriverVersion"]}") - implementation("com.clickhouse:clickhouse-data:${project.extra["clickHouseDriverVersion"]}") - implementation("com.clickhouse:client-v2:${project.extra["clickHouseDriverVersion"]}") + implementation("com.clickhouse:client-v2:${project.extra["clickHouseDriverVersion"]}:all") // Apache Flink Libraries implementation("org.apache.flink:flink-connector-base:${project.extra["flinkVersion"]}") implementation("org.apache.flink:flink-streaming-java:${project.extra["flinkVersion"]}") diff --git a/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java b/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java new file mode 100644 index 0000000..7f0c7bf --- /dev/null +++ b/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java @@ -0,0 +1,183 @@ +package com.clickhouse.utils; + +import com.clickhouse.client.api.data_formats.internal.SerializerUtils; +import com.clickhouse.data.ClickHouseDataType; +import com.clickhouse.data.format.BinaryStreamUtils; + +import java.io.IOException; +import java.io.OutputStream; +import java.lang.reflect.Method; +import java.time.LocalDate; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.util.HashMap; +import java.util.Map; + +//import static com.clickhouse.client.api.data_formats.internal.SerializerUtils.writeDate; + +public class Serialize { + + public static boolean writePrimitiveValuePreamble(OutputStream out, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column) throws IOException { + // since it is primitive we always have a value that is not null + if (defaultsSupport) { + SerializerUtils.writeNonNull(out); + if (isNullable) { + SerializerUtils.writeNonNull(out); + } + } else if (isNullable) { + SerializerUtils.writeNonNull(out); + } + return true; + } + public static boolean writeValuePreamble(OutputStream out, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column, Object value) throws IOException { + if (defaultsSupport) { + if (value != null) { + SerializerUtils.writeNonNull(out); + if (isNullable) { + SerializerUtils.writeNonNull(out); + } + } else { + if (hasDefault) { + SerializerUtils.writeNull(out); + return false; + } + + if (isNullable) { + SerializerUtils.writeNonNull(out); + SerializerUtils.writeNull(out); + return false; + } + + if (dataType == ClickHouseDataType.Array) { + SerializerUtils.writeNonNull(out); + } else if (dataType != ClickHouseDataType.Dynamic) { + throw new IllegalArgumentException(String.format("An attempt to write null into not nullable column '%s'", column)); + } + } + } else if (isNullable) { + if (value == null) { + SerializerUtils.writeNull(out); + return false; + } + + SerializerUtils.writeNonNull(out); + } else if (value == null) { + if (dataType == ClickHouseDataType.Array) { + SerializerUtils.writeNonNull(out); + } else if (dataType != ClickHouseDataType.Dynamic) { + throw new IllegalArgumentException(String.format("An attempt to write null into not nullable column '%s'", column)); + } + } + return true; + } + + public static String convertToString(Object value) { + return java.lang.String.valueOf(value); + } + + public static Integer convertToInteger(Object value) { + if (value instanceof Integer) { + return (Integer) value; + } else if (value instanceof Number) { + return ((Number) value).intValue(); + } else if (value instanceof String) { + return Integer.parseInt((String) value); + } else if (value instanceof Boolean) { + return ((Boolean) value) ? 1 : 0; + } else { + throw new IllegalArgumentException("Cannot convert " + value + " to Integer"); + } + } + + public static Map mapClickHouseTypeToMethod() { + Map map = new HashMap<>(); + for (Method method : Serialize.class.getMethods()) { + String name = method.getName(); + if (name.startsWith("write")) { + String chType = name.substring("write".length()); + try { + ClickHouseDataType type = ClickHouseDataType.valueOf(chType); + map.put(type, method); + } catch (IllegalArgumentException e) { + System.out.println(e.getMessage()); + } + } + } + return map; + } + + /** + * + */ + + // Method structure write[ClickHouse Type](OutputStream, Java type, ... ) + // Date support + public static void writeDate(OutputStream out, LocalDate value, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column) throws IOException { + System.out.println("writeDate"); + if (writeValuePreamble(out, defaultsSupport, isNullable, dataType, hasDefault, column, value)) { + SerializerUtils.writeDate(out, value, ZoneId.of("UTC")); // TODO: check + } + } + + public static void writeDate(OutputStream out, ZonedDateTime value, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column) throws IOException { + if (writeValuePreamble(out, defaultsSupport, isNullable, dataType, hasDefault, column, value)) { + SerializerUtils.writeDate(out, value, ZoneId.of("UTC")); // TODO: check + } + } + + // clickhouse type String support + public static void writeString(OutputStream out, String value, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column) throws IOException { + if (writeValuePreamble(out, defaultsSupport, isNullable, dataType, hasDefault, column, value)) { + BinaryStreamUtils.writeString(out, convertToString(value)); + } + } + + public static void writeFixedString(OutputStream out, String value, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, int size, String column) throws IOException { + if (writeValuePreamble(out, defaultsSupport, isNullable, dataType, hasDefault, column, value)) { + BinaryStreamUtils.writeFixedString(out, convertToString(value), size); + } + } + + // Int8 + public static void writeInt8(OutputStream out, Byte value, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column) throws IOException { + if (writeValuePreamble(out, defaultsSupport, isNullable, dataType, hasDefault, column, value)) { + BinaryStreamUtils.writeInt8(out, convertToInteger(value)); + } + } + + // Int16 + public static void writeInt16(OutputStream out, Short value, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column) throws IOException { + if (writeValuePreamble(out, defaultsSupport, isNullable, dataType, hasDefault, column, value)) { + BinaryStreamUtils.writeInt16(out, convertToInteger(value)); + } + } + + // Int32 + public static void writeInt32(OutputStream out, Integer value, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column) throws IOException { + if (writeValuePreamble(out, defaultsSupport, isNullable, dataType, hasDefault, column, value)) { + BinaryStreamUtils.writeInt32(out, convertToInteger(value)); + } + } + + // Int64 + public static void writeInt64(OutputStream out, Long value, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column) throws IOException { + if (writeValuePreamble(out, defaultsSupport, isNullable, dataType, hasDefault, column, value)) { + BinaryStreamUtils.writeInt64(out, convertToInteger(value)); + } + } + + // Float32 + public static void writeFloat32(OutputStream out, Float value, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column) throws IOException { + if (writeValuePreamble(out, defaultsSupport, isNullable, dataType, hasDefault, column, value)) { + BinaryStreamUtils.writeFloat32(out, value); + } + } + + // Float64 + public static void writeFloat64(OutputStream out, Double value, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column) throws IOException { + if (writeValuePreamble(out, defaultsSupport, isNullable, dataType, hasDefault, column, value)) { + BinaryStreamUtils.writeFloat64(out, value); + } + } + +} diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java index 14b644d..c579527 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java @@ -7,9 +7,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; + public class ClickHouseConvertor implements ElementConverter { private static final Logger LOG = LoggerFactory.getLogger(ClickHouseConvertor.class); + POJOConvertor pojoConvertor = null; enum Types { STRING, POJO, @@ -30,6 +32,15 @@ public ClickHouseConvertor(Class clazz) { } } + public ClickHouseConvertor(Class clazz, POJOConvertor pojoConvertor) { + if (clazz == null) { + throw new IllegalArgumentException("clazz must not be not null"); + } else { + type = Types.POJO; + this.pojoConvertor = pojoConvertor; + } + } + @Override public ClickHousePayload apply( InputT o, SinkWriter.Context context) { if (o == null) { @@ -47,8 +58,13 @@ public ClickHousePayload apply( InputT o, SinkWriter.Context context) { return new ClickHousePayload((payload + "\n").getBytes()); } if (type == Types.POJO) { - // TODO Convert to byte stream - return null; + // TODO Convert POJO to bytes + try { + byte[] payload = this.pojoConvertor.convert(o); + return new ClickHousePayload(payload); + } catch (Exception e) { + return new ClickHousePayload(null); + } } throw new IllegalArgumentException("unable to convert " + o + " to " + type); } diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/POJOConvertor.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/POJOConvertor.java new file mode 100644 index 0000000..3cd0da5 --- /dev/null +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/POJOConvertor.java @@ -0,0 +1,16 @@ +package org.apache.flink.connector.clickhouse.convertor; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.io.Serializable; + +public abstract class POJOConvertor implements Serializable { + public abstract void instrument(OutputStream out, InputT input) throws IOException; + + public byte[] convert(InputT input) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + instrument(out, input); + return out.toByteArray(); + } +} diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/data/ClickHousePayload.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/data/ClickHousePayload.java index c35e211..d3305f4 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/data/ClickHousePayload.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/data/ClickHousePayload.java @@ -1,6 +1,5 @@ package org.apache.flink.connector.clickhouse.data; -import org.apache.flink.connector.base.sink.writer.AsyncSinkWriterStateSerializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java index 852e9a9..3cc687c 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java @@ -2,7 +2,6 @@ import com.clickhouse.client.api.Client; import com.clickhouse.client.api.ClientConfigProperties; -import org.apache.flink.connector.clickhouse.data.ClickHousePayload; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -16,6 +15,8 @@ public class ClickHouseClientConfig implements Serializable { private final String password; private final String database; private final String tableName; +// private List> classToReisterList = null; +// private List tableSchemaList = null; public ClickHouseClientConfig(String url, String username, String password, String database, String tableName) { this.url = url; @@ -23,18 +24,31 @@ public ClickHouseClientConfig(String url, String username, String password, Stri this.password = password; this.database = database; this.tableName = tableName; +// this.classToReisterList = new ArrayList<>(); +// this.tableSchemaList = new ArrayList<>(); } public Client createClient(String database) { - return new Client.Builder() + Client client = new Client.Builder() .addEndpoint(url) .setUsername(username) .setPassword(password) .setDefaultDatabase(database) .setOption(ClientConfigProperties.ASYNC_OPERATIONS.getKey(), "true") .build(); +// if (classToReisterList != null) { +// for (int index = 0; index < classToReisterList.size(); index++) { +// client.register(classToReisterList.get(index), tableSchemaList.get(index)); +// } +// } + return client; } +// public void registerClass(Class clazz, TableSchema tableSchema) { +// classToReisterList.add(clazz); +// tableSchemaList.add(tableSchema); +// } + public Client createClient() { return createClient(this.database); } diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java index cad95ed..7e59f01 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java @@ -1,26 +1,32 @@ package org.apache.flink.connector.clickhouse.sink; +import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.data.ClickHouseFormat; -import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.connector.base.sink.writer.ElementConverter; import org.apache.flink.connector.clickhouse.convertor.ClickHouseConvertor; +import org.apache.flink.connector.clickhouse.convertor.POJOConvertor; import org.apache.flink.connector.clickhouse.data.ClickHousePayload; +import org.apache.flink.connector.clickhouse.sink.convertor.CovidPOJOConvertor; +import org.apache.flink.connector.clickhouse.sink.convertor.SimplePOJOConvertor; +import org.apache.flink.connector.clickhouse.sink.pojo.CovidPOJO; +import org.apache.flink.connector.clickhouse.sink.pojo.SimplePOJO; +import org.apache.flink.connector.file.src.FileSource; +import org.apache.flink.connector.file.src.reader.TextLineInputFormat; import org.apache.flink.connector.test.FlinkClusterTests; import org.apache.flink.connector.test.embedded.clickhouse.ClickHouseServerForTests; import org.apache.flink.connector.test.embedded.flink.EmbeddedFlinkClusterForTests; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.connector.file.src.FileSource; -import org.apache.flink.connector.file.src.reader.TextLineInputFormat; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.net.URL; -import java.nio.file.Paths; -import java.util.concurrent.ExecutionException; +import java.util.ArrayList; +import java.util.List; public class ClickHouseSinkTests extends FlinkClusterTests { @@ -45,7 +51,6 @@ private int executeJob(StreamExecutionEnvironment env, String tableName) throws @Test void CSVDataTest() throws Exception { - //String path = ClickHouseSinkTests.class.getClassLoader().getResource(".").toString(); String tableName = "csv_covid"; String dropTable = String.format("DROP TABLE IF EXISTS `%s`.`%s`", getDatabase(), tableName); ClickHouseServerForTests.executeSql(dropTable); @@ -65,6 +70,7 @@ void CSVDataTest() throws Exception { "ENGINE = MergeTree " + "ORDER BY (location_key, date); "; ClickHouseServerForTests.executeSql(tableSql); + final StreamExecutionEnvironment env = EmbeddedFlinkClusterForTests.getMiniCluster().getTestStreamEnvironment(); env.setParallelism(1); @@ -81,7 +87,6 @@ void CSVDataTest() throws Exception { 1000, clickHouseClientConfig ); - // in case of just want to forward our data use the appropriate ClickHouse format csvSink.setClickHouseFormat(ClickHouseFormat.CSV); Path filePath = new Path("./src/test/resources/epidemiology_top_10000.csv.gz"); @@ -89,7 +94,7 @@ void CSVDataTest() throws Exception { FileSource source = FileSource .forRecordStreamFormat(new TextLineInputFormat(), filePath) .build(); - + // read csv data from file DataStreamSource lines = env.fromSource( source, WatermarkStrategy.noWatermarks(), @@ -99,4 +104,138 @@ void CSVDataTest() throws Exception { int rows = executeJob(env, tableName); Assertions.assertEquals(EXPECTED_ROWS, rows); } + + @Test + void CovidPOJODataTest() throws Exception { + String tableName = "covid_pojo"; + + String dropTable = String.format("DROP TABLE IF EXISTS `%s`.`%s`", getDatabase(), tableName); + ClickHouseServerForTests.executeSql(dropTable); + // create table + String tableSql = "CREATE TABLE `" + getDatabase() + "`.`" + tableName + "` (" + + "date Date," + + "location_key LowCardinality(String)," + + "new_confirmed Int32," + + "new_deceased Int32," + + "new_recovered Int32," + + "new_tested Int32," + + "cumulative_confirmed Int32," + + "cumulative_deceased Int32," + + "cumulative_recovered Int32," + + "cumulative_tested Int32" + + ") " + + "ENGINE = MergeTree " + + "ORDER BY (location_key, date); "; + ClickHouseServerForTests.executeSql(tableSql); + + + TableSchema covidTableSchema = ClickHouseServerForTests.getTableSchema(tableName); +// POJOConvertor covidPOJOConvertor = POJOSerializable.create().createConvertor(covidTableSchema, CovidPOJO.class); + + POJOConvertor covidPOJOConvertor = new CovidPOJOConvertor(); + final StreamExecutionEnvironment env = EmbeddedFlinkClusterForTests.getMiniCluster().getTestStreamEnvironment(); + env.setParallelism(5); + + ClickHouseClientConfig clickHouseClientConfig = new ClickHouseClientConfig(getServerURL(), getUsername(), getPassword(), getDatabase(), tableName); + ElementConverter convertorCovid = new ClickHouseConvertor<>(CovidPOJO.class, covidPOJOConvertor); + + ClickHouseAsyncSink covidPOJOSink = new ClickHouseAsyncSink<>( + convertorCovid, + 5000, + 2, + 20000, + 1024 * 1024, + 5 * 1000, + 1000, + clickHouseClientConfig + ); + + covidPOJOSink.setClickHouseFormat(ClickHouseFormat.RowBinary); + + Path filePath = new Path("./src/test/resources/epidemiology_top_10000.csv.gz"); + + FileSource source = FileSource + .forRecordStreamFormat(new TextLineInputFormat(), filePath) + .build(); + // read csv data from file + DataStreamSource lines = env.fromSource( + source, + WatermarkStrategy.noWatermarks(), + "GzipCsvSource" + ); + + // convert to POJO + DataStream covidPOJOs = lines.map(new MapFunction() { + @Override + public CovidPOJO map(String value) throws Exception { + return new CovidPOJO(value); + } + }); + // send to a sink + covidPOJOs.sinkTo(covidPOJOSink); + int rows = executeJob(env, tableName); + Assertions.assertEquals(EXPECTED_ROWS, rows); + } + + @Test + void SimplePOJODataTest() throws Exception { + // TODO: needs to be extended to all types + String tableName = "simple_pojo"; + + String dropTable = String.format("DROP TABLE IF EXISTS `%s`.`%s`", getDatabase(), tableName); + ClickHouseServerForTests.executeSql(dropTable); + // create table + String tableSql = "CREATE TABLE `" + getDatabase() + "`.`" + tableName + "` (" + + "bytePrimitive Int8," + + "byteObject Int8," + + "shortPrimitive Int16," + + "shortObject Int16," + + "intPrimitive Int32," + + "integerObject Int32," + + "longPrimitive Int64," + + "longObject Int64," + + "floatPrimitive Float," + + "floatObject Float," + + "doublePrimitive Double," + + "doubleObject Double," + + ") " + + "ENGINE = MergeTree " + + "ORDER BY (longPrimitive); "; + ClickHouseServerForTests.executeSql(tableSql); + + + TableSchema simpleTableSchema = ClickHouseServerForTests.getTableSchema(tableName); +// POJOConvertor simplePOJOConvertor = POJOSerializable.create().createConvertor(simpleTableSchema, SimplePOJO.class); + POJOConvertor simplePOJOConvertor = new SimplePOJOConvertor(); + + final StreamExecutionEnvironment env = EmbeddedFlinkClusterForTests.getMiniCluster().getTestStreamEnvironment(); + env.setParallelism(5); + + ClickHouseClientConfig clickHouseClientConfig = new ClickHouseClientConfig(getServerURL(), getUsername(), getPassword(), getDatabase(), tableName); + ElementConverter convertorCovid = new ClickHouseConvertor<>(CovidPOJO.class, simplePOJOConvertor); + + ClickHouseAsyncSink simplePOJOSink = new ClickHouseAsyncSink<>( + convertorCovid, + 5000, + 2, + 20000, + 1024 * 1024, + 5 * 1000, + 1000, + clickHouseClientConfig + ); + + simplePOJOSink.setClickHouseFormat(ClickHouseFormat.RowBinary); + + List simplePOJOList = new ArrayList<>(); + for (int i = 0; i < EXPECTED_ROWS; i++) { + simplePOJOList.add(new SimplePOJO()); + } + // create from list + DataStream simplePOJOs = env.fromData(simplePOJOList.toArray(new SimplePOJO[0])); + // send to a sink + simplePOJOs.sinkTo(simplePOJOSink); + int rows = executeJob(env, tableName); + Assertions.assertEquals(EXPECTED_ROWS, rows); + } } diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/convertor/CovidPOJOConvertor.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/convertor/CovidPOJOConvertor.java new file mode 100644 index 0000000..55c148b --- /dev/null +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/convertor/CovidPOJOConvertor.java @@ -0,0 +1,25 @@ +package org.apache.flink.connector.clickhouse.sink.convertor; + +import com.clickhouse.data.ClickHouseDataType; +import com.clickhouse.utils.Serialize; +import org.apache.flink.connector.clickhouse.convertor.POJOConvertor; +import org.apache.flink.connector.clickhouse.sink.pojo.CovidPOJO; + +import java.io.IOException; +import java.io.OutputStream; + +public class CovidPOJOConvertor extends POJOConvertor { + @Override + public void instrument(OutputStream out, CovidPOJO input) throws IOException { + Serialize.writeDate(out, input.getLocalDate(), false, false, ClickHouseDataType.Date, false, "date"); + Serialize.writeString(out, input.getLocation_key(), false, false, ClickHouseDataType.String, false, "location_key"); + Serialize.writeInt32(out, input.getNew_confirmed(), false, false, ClickHouseDataType.Int32, false, "new_confirmed"); + Serialize.writeInt32(out, input.getNew_deceased(), false, false, ClickHouseDataType.Int32, false, ""); + Serialize.writeInt32(out, input.getNew_recovered(), false, false, ClickHouseDataType.Int32, false, ""); + Serialize.writeInt32(out, input.getNew_tested(), false, false, ClickHouseDataType.Int32, false, ""); + Serialize.writeInt32(out, input.getCumulative_confirmed(), false, false, ClickHouseDataType.Int32, false, ""); + Serialize.writeInt32(out, input.getCumulative_deceased(), false, false, ClickHouseDataType.Int32, false, ""); + Serialize.writeInt32(out, input.getCumulative_recovered(), false, false, ClickHouseDataType.Int32, false, ""); + Serialize.writeInt32(out, input.getCumulative_tested(), false, false, ClickHouseDataType.Int32, false, ""); + } +} diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/convertor/SimplePOJOConvertor.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/convertor/SimplePOJOConvertor.java new file mode 100644 index 0000000..4b5cb67 --- /dev/null +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/convertor/SimplePOJOConvertor.java @@ -0,0 +1,32 @@ +package org.apache.flink.connector.clickhouse.sink.convertor; + +import com.clickhouse.data.ClickHouseDataType; +import com.clickhouse.utils.Serialize; +import org.apache.flink.connector.clickhouse.convertor.POJOConvertor; +import org.apache.flink.connector.clickhouse.sink.pojo.SimplePOJO; + +import java.io.IOException; +import java.io.OutputStream; + +public class SimplePOJOConvertor extends POJOConvertor { + @Override + public void instrument(OutputStream out, SimplePOJO input) throws IOException { + Serialize.writeInt8(out, input.getBytePrimitive(), false, false, ClickHouseDataType.Int8, false, "bytePrimitive"); + Serialize.writeInt8(out, input.getByteObject(), false, false, ClickHouseDataType.Int8, false, "byteObject"); + + Serialize.writeInt16(out, input.getShortPrimitive(), false, false, ClickHouseDataType.Int16, false, "shortPrimitive"); + Serialize.writeInt16(out, input.getShortObject(), false, false, ClickHouseDataType.Int16, false, "shortObject"); + + Serialize.writeInt32(out, input.getIntPrimitive(), false, false, ClickHouseDataType.Int32, false, "intPrimitive"); + Serialize.writeInt32(out, input.getIntegerObject(), false, false, ClickHouseDataType.Int32, false, "integerObject"); + + Serialize.writeInt64(out, input.getLongPrimitive(), false, false, ClickHouseDataType.Int64, false, "longPrimitive"); + Serialize.writeInt64(out, input.getLongObject(), false, false, ClickHouseDataType.Int64, false, "longObject"); + + Serialize.writeFloat32(out, input.getFloatPrimitive(), false, false, ClickHouseDataType.Float32, false, "floatPrimitive"); + Serialize.writeFloat32(out, input.getFloatObject(), false, false, ClickHouseDataType.Float32, false, "floatObject"); + + Serialize.writeFloat64(out, input.getDoublePrimitive(), false, false, ClickHouseDataType.Float64, false, "doublePrimitive"); + Serialize.writeFloat64(out, input.getDoubleObject(), false, false, ClickHouseDataType.Float64, false, "doubleObject"); + } +} diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/pojo/CovidPOJO.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/pojo/CovidPOJO.java new file mode 100644 index 0000000..5f9bc89 --- /dev/null +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/pojo/CovidPOJO.java @@ -0,0 +1,101 @@ +package org.apache.flink.connector.clickhouse.sink.pojo; + +import java.time.LocalDate; +import java.time.format.DateTimeFormatter; + +public class CovidPOJO { + private LocalDate date; + private String location_key; + private Integer new_confirmed; + private Integer new_deceased; + private Integer new_recovered; + private Integer new_tested; + private Integer cumulative_confirmed; + private Integer cumulative_deceased; + private Integer cumulative_recovered; + private Integer cumulative_tested; + static DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); + + public CovidPOJO(String payload) { + String[] values = payload.split(","); + this.date = LocalDate.parse(values[0].trim(), formatter); + this.location_key = values[1].trim(); + this.new_confirmed = valueOf(values[2].trim()); + this.new_deceased = valueOf(values[3].trim()); + this.new_recovered = valueOf(values[4].trim()); + this.new_tested = valueOf(values[5].trim()); + this.cumulative_confirmed = valueOf(values[6].trim()); + this.cumulative_deceased = valueOf(values[7].trim()); + this.cumulative_recovered = 0; + this.cumulative_tested = 0; + } + + private Integer valueOf(String value) { + if (value == null || value.trim().isEmpty()) { + return 0; + } else { + return Integer.valueOf(value); + } + } + + public LocalDate getLocalDate() { + return date; + } + public void setDate(LocalDate date) { + this.date = date; + } + public String getLocation_key() { + return location_key; + } + public void setLocation_key(String location_key) { + this.location_key = location_key; + } + public Integer getNew_confirmed() { + return new_confirmed; + } + public void setNew_confirmed(Integer new_confirmed) { + this.new_confirmed = new_confirmed; + } + public Integer getNew_deceased() { + return new_deceased; + } + public void setNew_deceased(Integer new_deceased) { + this.new_deceased = new_deceased; + } + public Integer getNew_recovered() { + return new_recovered; + } + public void setNew_recovered(Integer new_recovered) { + this.new_recovered = new_recovered; + } + public Integer getNew_tested() { + return new_tested; + } + public void setNew_tested(Integer new_tested) { + this.new_tested = new_tested; + } + public Integer getCumulative_confirmed() { + return cumulative_confirmed; + } + public void setCumulative_confirmed(Integer cumulative_confirmed) { + this.cumulative_confirmed = cumulative_confirmed; + } + public Integer getCumulative_deceased() { + return cumulative_deceased; + } + public void setCumulative_deceased(Integer cumulative_deceased) { + this.cumulative_deceased = cumulative_deceased; + } + public Integer getCumulative_recovered() { + return cumulative_recovered; + } + public void setCumulative_recovered(Integer cumulative_recovered) { + this.cumulative_recovered = cumulative_recovered; + } + public Integer getCumulative_tested() { + return cumulative_tested; + } + public void setCumulative_tested(Integer cumulative_tested) { + this.cumulative_tested = cumulative_tested; + } +} diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/pojo/SimplePOJO.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/pojo/SimplePOJO.java new file mode 100644 index 0000000..4b203af --- /dev/null +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/pojo/SimplePOJO.java @@ -0,0 +1,90 @@ +package org.apache.flink.connector.clickhouse.sink.pojo; + +public class SimplePOJO { + + private byte bytePrimitive; + private Byte byteObject; + + private short shortPrimitive; + private Short shortObject; + + private int intPrimitive; + private Integer integerObject; + + private long longPrimitive; + private Long longObject; + + private float floatPrimitive; + private Float floatObject; + + private double doublePrimitive; + private Double doubleObject; + + public SimplePOJO() { + this.bytePrimitive = Byte.MIN_VALUE; + this.byteObject = Byte.MAX_VALUE; + + this.shortPrimitive = Short.MIN_VALUE; + this.shortObject = Short.MAX_VALUE; + + this.intPrimitive = Integer.MIN_VALUE; + this.integerObject = Integer.MAX_VALUE; + + this.longPrimitive = Long.MIN_VALUE; + this.longObject = Long.MAX_VALUE; + + this.floatPrimitive = Float.MIN_VALUE; + this.floatObject = Float.MAX_VALUE; + + this.doublePrimitive = Double.MIN_VALUE; + this.doubleObject = Double.MAX_VALUE; + } + + public byte getBytePrimitive() { + return bytePrimitive; + } + + public Byte getByteObject() { + return byteObject; + } + + public short getShortPrimitive() { + return shortPrimitive; + } + + public Short getShortObject() { + return shortObject; + } + + public int getIntPrimitive() { + return intPrimitive; + } + + public Integer getIntegerObject() { + return integerObject; + } + + public long getLongPrimitive() { + return longPrimitive; + } + + public Long getLongObject() { + return longObject; + } + + public float getFloatPrimitive() { + return floatPrimitive; + } + + public Float getFloatObject() { + return floatObject; + } + + public double getDoublePrimitive() { + return doublePrimitive; + } + + public Double getDoubleObject() { + return doubleObject; + } +} diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/DummyFlinkClusterTest.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/DummyFlinkClusterTest.java index 6067a2b..c9cb535 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/DummyFlinkClusterTest.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/DummyFlinkClusterTest.java @@ -6,7 +6,8 @@ import org.apache.flink.streaming.api.functions.sink.legacy.SinkFunction; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.junit.jupiter.api.*; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Collections; diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/embedded/clickhouse/ClickHouseServerForTests.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/embedded/clickhouse/ClickHouseServerForTests.java index 50d5fa8..ff4b0fd 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/embedded/clickhouse/ClickHouseServerForTests.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/embedded/clickhouse/ClickHouseServerForTests.java @@ -1,18 +1,15 @@ package org.apache.flink.connector.test.embedded.clickhouse; import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.GenericRecord; -import org.apache.flink.streaming.api.functions.sink.legacy.SinkFunction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testcontainers.clickhouse.ClickHouseContainer; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.concurrent.ExecutionException; -import org.testcontainers.clickhouse.ClickHouseContainer; - public class ClickHouseServerForTests { private static final Logger LOG = LoggerFactory.getLogger(ClickHouseServerForTests.class); @@ -100,4 +97,9 @@ public static int countRows(String table) throws ExecutionException, Interrupted List countResult = client.queryAll(countSql); return countResult.get(0).getInteger(1); } + + public static TableSchema getTableSchema(String table) throws ExecutionException, InterruptedException { + Client client = ClickHouseTestHelpers.getClient(host, port, isSSL, username, password); + return client.getTableSchema(table, database); + } } diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/embedded/flink/EmbeddedFlinkClusterForTests.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/embedded/flink/EmbeddedFlinkClusterForTests.java index 63a8447..8030164 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/embedded/flink/EmbeddedFlinkClusterForTests.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/test/embedded/flink/EmbeddedFlinkClusterForTests.java @@ -5,8 +5,6 @@ import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; public class EmbeddedFlinkClusterForTests { From f5d286e2702990dff03b5bd85c66f3839e0a3235 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Mon, 2 Jun 2025 12:16:50 +0300 Subject: [PATCH 07/23] Update Jave client version. --- flink-connector-clickhouse-base/build.gradle.kts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connector-clickhouse-base/build.gradle.kts b/flink-connector-clickhouse-base/build.gradle.kts index 8aa5870..6a6d7d1 100644 --- a/flink-connector-clickhouse-base/build.gradle.kts +++ b/flink-connector-clickhouse-base/build.gradle.kts @@ -17,7 +17,7 @@ repositories { } extra.apply { - set("clickHouseDriverVersion", "0.8.6-SNAPSHOT") + set("clickHouseDriverVersion", "0.8.6") set("flinkVersion", "2.0.0") set("log4jVersion","2.17.2") set("testContainersVersion", "1.21.0") From 94a85dba51b62fff27b1f0c060db93336107efa6 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Mon, 2 Jun 2025 14:01:38 +0300 Subject: [PATCH 08/23] Fix SimplePOJO content to avoid dedup --- .../flink/connector/clickhouse/sink/ClickHouseSinkTests.java | 2 +- .../flink/connector/clickhouse/sink/pojo/SimplePOJO.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java index 7e59f01..1a9ea5f 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java @@ -229,7 +229,7 @@ void SimplePOJODataTest() throws Exception { List simplePOJOList = new ArrayList<>(); for (int i = 0; i < EXPECTED_ROWS; i++) { - simplePOJOList.add(new SimplePOJO()); + simplePOJOList.add(new SimplePOJO(i)); } // create from list DataStream simplePOJOs = env.fromData(simplePOJOList.toArray(new SimplePOJO[0])); diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/pojo/SimplePOJO.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/pojo/SimplePOJO.java index 4b203af..06a0881 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/pojo/SimplePOJO.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/pojo/SimplePOJO.java @@ -20,7 +20,7 @@ public class SimplePOJO { private double doublePrimitive; private Double doubleObject; - public SimplePOJO() { + public SimplePOJO(int index) { this.bytePrimitive = Byte.MIN_VALUE; this.byteObject = Byte.MAX_VALUE; @@ -30,7 +30,7 @@ public SimplePOJO() { this.intPrimitive = Integer.MIN_VALUE; this.integerObject = Integer.MAX_VALUE; - this.longPrimitive = Long.MIN_VALUE; + this.longPrimitive = index; this.longObject = Long.MAX_VALUE; this.floatPrimitive = Float.MIN_VALUE; From ed92b0a2ef8de0c8af217008ba3b2d75133b8a13 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Mon, 2 Jun 2025 14:48:22 +0300 Subject: [PATCH 09/23] Remove System.out --- .../flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java index dfb7517..12755ea 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java @@ -60,7 +60,6 @@ protected long getSizeInBytes(ClickHousePayload clickHousePayload) { @Override protected void submitRequestEntries(List requestEntries, ResultHandler resultHandler) { LOG.info("Submitting request entries..."); - System.out.println("Submitting request entries..."); AtomicInteger totalSizeSend = new AtomicInteger(); Client chClient = this.clickHouseClientConfig.createClient(); String tableName = clickHouseClientConfig.getTableName(); From 1adbe8432b10ac6e6135737c02ddce6df842502e Mon Sep 17 00:00:00 2001 From: Mark Zitnik Date: Mon, 2 Jun 2025 15:02:26 +0300 Subject: [PATCH 10/23] Update flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/convertor/CovidPOJOConvertor.java Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- .../sink/convertor/CovidPOJOConvertor.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/convertor/CovidPOJOConvertor.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/convertor/CovidPOJOConvertor.java index 55c148b..51f47df 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/convertor/CovidPOJOConvertor.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/convertor/CovidPOJOConvertor.java @@ -14,12 +14,12 @@ public void instrument(OutputStream out, CovidPOJO input) throws IOException { Serialize.writeDate(out, input.getLocalDate(), false, false, ClickHouseDataType.Date, false, "date"); Serialize.writeString(out, input.getLocation_key(), false, false, ClickHouseDataType.String, false, "location_key"); Serialize.writeInt32(out, input.getNew_confirmed(), false, false, ClickHouseDataType.Int32, false, "new_confirmed"); - Serialize.writeInt32(out, input.getNew_deceased(), false, false, ClickHouseDataType.Int32, false, ""); - Serialize.writeInt32(out, input.getNew_recovered(), false, false, ClickHouseDataType.Int32, false, ""); - Serialize.writeInt32(out, input.getNew_tested(), false, false, ClickHouseDataType.Int32, false, ""); - Serialize.writeInt32(out, input.getCumulative_confirmed(), false, false, ClickHouseDataType.Int32, false, ""); - Serialize.writeInt32(out, input.getCumulative_deceased(), false, false, ClickHouseDataType.Int32, false, ""); - Serialize.writeInt32(out, input.getCumulative_recovered(), false, false, ClickHouseDataType.Int32, false, ""); - Serialize.writeInt32(out, input.getCumulative_tested(), false, false, ClickHouseDataType.Int32, false, ""); + Serialize.writeInt32(out, input.getNew_deceased(), false, false, ClickHouseDataType.Int32, false, "new_deceased"); + Serialize.writeInt32(out, input.getNew_recovered(), false, false, ClickHouseDataType.Int32, false, "new_recovered"); + Serialize.writeInt32(out, input.getNew_tested(), false, false, ClickHouseDataType.Int32, false, "new_tested"); + Serialize.writeInt32(out, input.getCumulative_confirmed(), false, false, ClickHouseDataType.Int32, false, "cumulative_confirmed"); + Serialize.writeInt32(out, input.getCumulative_deceased(), false, false, ClickHouseDataType.Int32, false, "cumulative_deceased"); + Serialize.writeInt32(out, input.getCumulative_recovered(), false, false, ClickHouseDataType.Int32, false, "cumulative_recovered"); + Serialize.writeInt32(out, input.getCumulative_tested(), false, false, ClickHouseDataType.Int32, false, "cumulative_tested"); } } From 6bdbf0babe8ad92c5a9b88824e823e5517dff513 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Mon, 2 Jun 2025 15:11:44 +0300 Subject: [PATCH 11/23] Fix copilot suggestions --- .../connector/clickhouse/convertor/ClickHouseConvertor.java | 4 +++- .../clickhouse/sink/ClickHouseAsyncSinkSerializer.java | 2 +- .../flink/connector/clickhouse/sink/ClickHouseSinkTests.java | 2 +- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java index c579527..6cf7c71 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java @@ -7,6 +7,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.nio.charset.StandardCharsets; + public class ClickHouseConvertor implements ElementConverter { private static final Logger LOG = LoggerFactory.getLogger(ClickHouseConvertor.class); @@ -54,7 +56,7 @@ public ClickHousePayload apply( InputT o, SinkWriter.Context context) { return new ClickHousePayload(null); } if (payload.endsWith("\n")) - return new ClickHousePayload(payload.getBytes()); + return new ClickHousePayload(payload.getBytes(StandardCharsets.UTF_8)); return new ClickHousePayload((payload + "\n").getBytes()); } if (type == Types.POJO) { diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSinkSerializer.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSinkSerializer.java index f7db1d5..be54c65 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSinkSerializer.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSinkSerializer.java @@ -10,7 +10,7 @@ import java.io.IOException; public class ClickHouseAsyncSinkSerializer extends AsyncSinkWriterStateSerializer { - private static final Logger LOG = LoggerFactory.getLogger(AsyncSinkWriterStateSerializer.class); + private static final Logger LOG = LoggerFactory.getLogger(ClickHouseAsyncSinkSerializer.class); @Override protected void serializeRequestToStream(ClickHousePayload clickHousePayload, DataOutputStream dataOutputStream) throws IOException { diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java index 1a9ea5f..35876ab 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java @@ -212,7 +212,7 @@ void SimplePOJODataTest() throws Exception { env.setParallelism(5); ClickHouseClientConfig clickHouseClientConfig = new ClickHouseClientConfig(getServerURL(), getUsername(), getPassword(), getDatabase(), tableName); - ElementConverter convertorCovid = new ClickHouseConvertor<>(CovidPOJO.class, simplePOJOConvertor); + ElementConverter convertorCovid = new ClickHouseConvertor<>(SimplePOJO.class, simplePOJOConvertor); ClickHouseAsyncSink simplePOJOSink = new ClickHouseAsyncSink<>( convertorCovid, From 11faf34adb0209d28d470383fc1abbc6cfca252a Mon Sep 17 00:00:00 2001 From: mzitnik Date: Tue, 3 Jun 2025 20:01:30 +0300 Subject: [PATCH 12/23] Added a few metrics & started to implement retry logic --- .../main/java/com/clickhouse/utils/Utils.java | 94 +++++++++++++++++++ .../exception/FlinkWriteException.java | 15 +++ .../exception/RetriableException.java | 13 +++ .../sink/ClickHouseAsyncWriter.java | 48 ++++++++-- .../sink/ClickHouseClientConfig.java | 21 ++--- .../clickhouse/sink/ClickHouseSinkTests.java | 5 +- 6 files changed, 171 insertions(+), 25 deletions(-) create mode 100644 flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Utils.java create mode 100644 flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/exception/FlinkWriteException.java create mode 100644 flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/exception/RetriableException.java diff --git a/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Utils.java b/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Utils.java new file mode 100644 index 0000000..7b5d5fd --- /dev/null +++ b/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Utils.java @@ -0,0 +1,94 @@ +package com.clickhouse.utils; + +import com.clickhouse.client.ClickHouseException; +import org.apache.flink.connector.clickhouse.exception.RetriableException; +import org.apache.flink.connector.clickhouse.sink.ClickHouseAsyncWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.SocketTimeoutException; +import java.net.UnknownHostException; +import java.util.Collection; + +public class Utils { + + private static final Logger LOG = LoggerFactory.getLogger(Utils.class); + private static final String CLICKHOUSE_CLIENT_ERROR_READ_TIMEOUT_MSG = "Read timed out after"; + private static final String CLICKHOUSE_CLIENT_ERROR_WRITE_TIMEOUT_MSG = "Write timed out after"; + + /** + * This will drill down to the first ClickHouseException in the exception chain + * + * @param e Exception to drill down + * @return ClickHouseException or null if none found + */ + public static Exception getRootCause(Throwable e, Boolean prioritizeClickHouseException) { + if (e == null) + return null; + + Throwable runningException = e;//We have to use Throwable because of the getCause() signature + while (runningException.getCause() != null && + (!prioritizeClickHouseException || !(runningException instanceof ClickHouseException))) { + LOG.trace("Found exception: {}", runningException.getLocalizedMessage()); + runningException = runningException.getCause(); + } + + return runningException instanceof Exception ? (Exception) runningException : null; + } + + /** + * This method checks to see if we should retry, otherwise it just throws the exception again + * + * @param e Exception to check + */ + + public static void handleException(Throwable e) { + LOG.warn("Deciding how to handle exception: {}", e.getLocalizedMessage()); + + //Let's check if we have a ClickHouseException to reference the error code + //https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/ErrorCodes.cpp + Exception rootCause = Utils.getRootCause(e, true); + if (rootCause instanceof ClickHouseException) { + ClickHouseException clickHouseException = (ClickHouseException) rootCause; + LOG.warn("ClickHouseException code: {}", clickHouseException.getErrorCode()); + switch (clickHouseException.getErrorCode()) { + case 3: // UNEXPECTED_END_OF_FILE + case 107: // FILE_DOESNT_EXIST + case 159: // TIMEOUT_EXCEEDED + case 164: // READONLY + case 202: // TOO_MANY_SIMULTANEOUS_QUERIES + case 203: // NO_FREE_CONNECTION + case 209: // SOCKET_TIMEOUT + case 210: // NETWORK_ERROR + case 241: // MEMORY_LIMIT_EXCEEDED + case 242: // TABLE_IS_READ_ONLY + case 252: // TOO_MANY_PARTS + case 285: // TOO_FEW_LIVE_REPLICAS + case 319: // UNKNOWN_STATUS_OF_INSERT + case 425: // SYSTEM_ERROR + case 999: // KEEPER_EXCEPTION + throw new RetriableException(e); + default: + LOG.error("Error code [{}] wasn't in the acceptable list.", clickHouseException.getErrorCode()); + break; + } + } + + //Otherwise use Root-Cause Exception Checking + if (rootCause instanceof SocketTimeoutException) { + LOG.warn("SocketTimeoutException thrown, wrapping exception: {}", e.getLocalizedMessage()); + throw new RetriableException(e); + } else if (rootCause instanceof UnknownHostException) { + LOG.warn("UnknownHostException thrown, wrapping exception: {}", e.getLocalizedMessage()); + throw new RetriableException(e); + } else if (rootCause instanceof IOException) { + final String msg = rootCause.getMessage(); + if (msg.indexOf(CLICKHOUSE_CLIENT_ERROR_READ_TIMEOUT_MSG) == 0 || msg.indexOf(CLICKHOUSE_CLIENT_ERROR_WRITE_TIMEOUT_MSG) == 0) { + LOG.warn("IOException thrown, wrapping exception: {}", e.getLocalizedMessage()); + throw new RetriableException(e); + } + } + } + +} diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/exception/FlinkWriteException.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/exception/FlinkWriteException.java new file mode 100644 index 0000000..9121125 --- /dev/null +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/exception/FlinkWriteException.java @@ -0,0 +1,15 @@ +package org.apache.flink.connector.clickhouse.exception; + +public class FlinkWriteException extends RuntimeException { + private static final long serialVersionUID = 1L; + + public FlinkWriteException(Throwable cause) { + + } + public FlinkWriteException(String message, Throwable cause) { + super(message, cause); + } + public FlinkWriteException(String message) { + super(message); + } +} diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/exception/RetriableException.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/exception/RetriableException.java new file mode 100644 index 0000000..4a85e3e --- /dev/null +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/exception/RetriableException.java @@ -0,0 +1,13 @@ +package org.apache.flink.connector.clickhouse.exception; + +public class RetriableException extends FlinkWriteException { + public RetriableException(String message) { + super(message); + } + public RetriableException(String message, Throwable cause) { + super(message, cause); + } + public RetriableException(Throwable cause) { + super(cause); + } +} diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java index 12755ea..902e43b 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java @@ -5,6 +5,7 @@ import com.clickhouse.client.api.insert.InsertResponse; import com.clickhouse.client.api.insert.InsertSettings; import com.clickhouse.data.ClickHouseFormat; +import com.clickhouse.utils.Utils; import org.apache.flink.api.connector.sink2.WriterInitContext; import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter; import org.apache.flink.connector.base.sink.writer.BufferedRequestState; @@ -12,6 +13,9 @@ import org.apache.flink.connector.base.sink.writer.ResultHandler; import org.apache.flink.connector.base.sink.writer.config.AsyncSinkWriterConfiguration; import org.apache.flink.connector.clickhouse.data.ClickHousePayload; +import org.apache.flink.connector.clickhouse.exception.RetriableException; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -26,6 +30,10 @@ public class ClickHouseAsyncWriter extends AsyncSinkWriter elementConverter, WriterInitContext context, int maxBatchSize, @@ -50,6 +58,10 @@ public ClickHouseAsyncWriter(ElementConverter element state); this.clickHouseClientConfig = clickHouseClientConfig; this.clickHouseFormat = clickHouseFormat; + final SinkWriterMetricGroup metricGroup = context.metricGroup(); + this.numBytesSendCounter = metricGroup.getNumBytesSendCounter(); + this.numRecordsSendCounter = metricGroup.getNumRecordsSendCounter(); + this.numRequestSubmittedCounter = metricGroup.counter("numRequestSubmitted"); } @Override @@ -59,25 +71,36 @@ protected long getSizeInBytes(ClickHousePayload clickHousePayload) { @Override protected void submitRequestEntries(List requestEntries, ResultHandler resultHandler) { + this.numRequestSubmittedCounter.inc(); LOG.info("Submitting request entries..."); - AtomicInteger totalSizeSend = new AtomicInteger(); Client chClient = this.clickHouseClientConfig.createClient(); String tableName = clickHouseClientConfig.getTableName(); // TODO: get from constructor or ClickHousePayload need to think what is the best way ClickHouseFormat format = null; - if (clickHouseFormat == null) { - // this not define lets try to get it from ClickHousePayload in case of POJO can be RowBinary or RowBinaryWithDefaults - } else { + if (clickHouseFormat != null) { format = clickHouseFormat; + } else { + // TODO: check if we configured payload to POJO serialization. + // this not define lets try to get it from ClickHousePayload in case of POJO can be RowBinary or RowBinaryWithDefaults + Boolean supportDefault = clickHouseClientConfig.getSupportDefault(); + if (supportDefault != null) { + if (supportDefault) format = ClickHouseFormat.RowBinaryWithDefaults; + else format = ClickHouseFormat.RowBinary; + } else { + throw new RuntimeException("ClickHouseFormat was not set "); + } } try { CompletableFuture response = chClient.insert(tableName, out -> { for (ClickHousePayload requestEntry : requestEntries) { byte[] payload = requestEntry.getPayload(); - totalSizeSend.addAndGet(payload.length); + // sum the data that is sent to ClickHouse + this.numBytesSendCounter.inc(payload.length); out.write(payload); } - LOG.info("Data that will be send to ClickHouse in bytes {} and the amount of records {}.", totalSizeSend.get(), requestEntries.size()); + // send the number that is sent to ClickHouse + this.numRecordsSendCounter.inc(requestEntries.size()); + LOG.info("Data that will be send to ClickHouse in bytes {} and the amount of records {}.", numBytesSendCounter.getCount(), requestEntries.size()); out.close(); }, format, new InsertSettings().setOption(ClientConfigProperties.ASYNC_OPERATIONS.getKey(), "true")); response.whenComplete((insertResponse, throwable) -> { @@ -108,9 +131,16 @@ private void handleFailedRequest( List requestEntries, ResultHandler resultHandler, Throwable error) { - // TODO extract from error if we can retry - error.printStackTrace(); - + // TODO: extract from error if we can retry + try { + Utils.handleException(error); + } catch (RetriableException e) { + LOG.info("Retriable exception occurred while processing request. ", e); + // TODO: send data again + resultHandler.retryForEntries(requestEntries); + } + LOG.info("completeExceptionally"); + resultHandler.completeExceptionally((Exception)error); } } diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java index 3cc687c..8e1f490 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java @@ -15,8 +15,7 @@ public class ClickHouseClientConfig implements Serializable { private final String password; private final String database; private final String tableName; -// private List> classToReisterList = null; -// private List tableSchemaList = null; + private Boolean supportDefault = null; public ClickHouseClientConfig(String url, String username, String password, String database, String tableName) { this.url = url; @@ -24,8 +23,6 @@ public ClickHouseClientConfig(String url, String username, String password, Stri this.password = password; this.database = database; this.tableName = tableName; -// this.classToReisterList = new ArrayList<>(); -// this.tableSchemaList = new ArrayList<>(); } public Client createClient(String database) { @@ -36,24 +33,20 @@ public Client createClient(String database) { .setDefaultDatabase(database) .setOption(ClientConfigProperties.ASYNC_OPERATIONS.getKey(), "true") .build(); -// if (classToReisterList != null) { -// for (int index = 0; index < classToReisterList.size(); index++) { -// client.register(classToReisterList.get(index), tableSchemaList.get(index)); -// } -// } return client; } -// public void registerClass(Class clazz, TableSchema tableSchema) { -// classToReisterList.add(clazz); -// tableSchemaList.add(tableSchema); -// } - public Client createClient() { return createClient(this.database); } public String getTableName() { return tableName; } + public void setSupportDefault(Boolean supportDefault) { + this.supportDefault = supportDefault; + } + public Boolean getSupportDefault() { + return supportDefault; + } } diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java index 35876ab..3556f08 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java @@ -137,6 +137,7 @@ void CovidPOJODataTest() throws Exception { env.setParallelism(5); ClickHouseClientConfig clickHouseClientConfig = new ClickHouseClientConfig(getServerURL(), getUsername(), getPassword(), getDatabase(), tableName); + clickHouseClientConfig.setSupportDefault(covidTableSchema.hasDefaults()); ElementConverter convertorCovid = new ClickHouseConvertor<>(CovidPOJO.class, covidPOJOConvertor); ClickHouseAsyncSink covidPOJOSink = new ClickHouseAsyncSink<>( @@ -150,8 +151,6 @@ void CovidPOJODataTest() throws Exception { clickHouseClientConfig ); - covidPOJOSink.setClickHouseFormat(ClickHouseFormat.RowBinary); - Path filePath = new Path("./src/test/resources/epidemiology_top_10000.csv.gz"); FileSource source = FileSource @@ -212,6 +211,8 @@ void SimplePOJODataTest() throws Exception { env.setParallelism(5); ClickHouseClientConfig clickHouseClientConfig = new ClickHouseClientConfig(getServerURL(), getUsername(), getPassword(), getDatabase(), tableName); + clickHouseClientConfig.setSupportDefault(simpleTableSchema.hasDefaults()); + ElementConverter convertorCovid = new ClickHouseConvertor<>(SimplePOJO.class, simplePOJOConvertor); ClickHouseAsyncSink simplePOJOSink = new ClickHouseAsyncSink<>( From 7875a7154ba24ffe191aaf2090d58748d5247163 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Thu, 5 Jun 2025 09:27:25 +0300 Subject: [PATCH 13/23] Remove remarked import --- .../src/main/java/com/clickhouse/utils/Serialize.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java b/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java index 7f0c7bf..adb4715 100644 --- a/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java +++ b/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java @@ -13,8 +13,6 @@ import java.util.HashMap; import java.util.Map; -//import static com.clickhouse.client.api.data_formats.internal.SerializerUtils.writeDate; - public class Serialize { public static boolean writePrimitiveValuePreamble(OutputStream out, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column) throws IOException { From 2bb4e81eae9bb395b63cdd3ea762f19d368ce22f Mon Sep 17 00:00:00 2001 From: mzitnik Date: Thu, 5 Jun 2025 09:34:35 +0300 Subject: [PATCH 14/23] remove uneeded code from writePrimitiveValuePreamble --- .../src/main/java/com/clickhouse/utils/Serialize.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java b/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java index adb4715..29582e7 100644 --- a/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java +++ b/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java @@ -18,11 +18,11 @@ public class Serialize { public static boolean writePrimitiveValuePreamble(OutputStream out, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column) throws IOException { // since it is primitive we always have a value that is not null if (defaultsSupport) { + // Add indicator since the table has default values SerializerUtils.writeNonNull(out); - if (isNullable) { - SerializerUtils.writeNonNull(out); - } - } else if (isNullable) { + } + // if the column is Nullable need to add an indicator for nullable + if (isNullable) { SerializerUtils.writeNonNull(out); } return true; From 4fc155bdacfe5622a61f6de5a435c25af5e64c57 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Thu, 5 Jun 2025 09:42:08 +0300 Subject: [PATCH 15/23] Added logging to writeValuePreamble --- .../src/main/java/com/clickhouse/utils/Serialize.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java b/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java index 29582e7..f959994 100644 --- a/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java +++ b/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java @@ -3,6 +3,7 @@ import com.clickhouse.client.api.data_formats.internal.SerializerUtils; import com.clickhouse.data.ClickHouseDataType; import com.clickhouse.data.format.BinaryStreamUtils; +import org.jline.utils.Log; import java.io.IOException; import java.io.OutputStream; @@ -28,6 +29,7 @@ public static boolean writePrimitiveValuePreamble(OutputStream out, boolean defa return true; } public static boolean writeValuePreamble(OutputStream out, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column, Object value) throws IOException { + Log.debug("writeValuePreamble[defaultsSupport='%s', isNullable='%s', dataType='%s', column='%s', value='%s']"); if (defaultsSupport) { if (value != null) { SerializerUtils.writeNonNull(out); @@ -49,7 +51,7 @@ public static boolean writeValuePreamble(OutputStream out, boolean defaultsSuppo if (dataType == ClickHouseDataType.Array) { SerializerUtils.writeNonNull(out); } else if (dataType != ClickHouseDataType.Dynamic) { - throw new IllegalArgumentException(String.format("An attempt to write null into not nullable column '%s'", column)); + throw new IllegalArgumentException(String.format("An attempt to write null into not nullable column '%s' of type '%s'", column, dataType)); } } } else if (isNullable) { @@ -63,7 +65,7 @@ public static boolean writeValuePreamble(OutputStream out, boolean defaultsSuppo if (dataType == ClickHouseDataType.Array) { SerializerUtils.writeNonNull(out); } else if (dataType != ClickHouseDataType.Dynamic) { - throw new IllegalArgumentException(String.format("An attempt to write null into not nullable column '%s'", column)); + throw new IllegalArgumentException(String.format("An attempt to write null into not nullable column '%s' of type '%s'", column, dataType)); } } return true; From 4dda9ca3e49617757a05fdf3424135caef08b04b Mon Sep 17 00:00:00 2001 From: mzitnik Date: Thu, 5 Jun 2025 10:07:45 +0300 Subject: [PATCH 16/23] Add logging & ignore if payload is null before sending --- .../clickhouse/convertor/ClickHouseConvertor.java | 1 + .../clickhouse/sink/ClickHouseAsyncWriter.java | 10 ++++++---- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java index 6cf7c71..822ea54 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java @@ -65,6 +65,7 @@ public ClickHousePayload apply( InputT o, SinkWriter.Context context) { byte[] payload = this.pojoConvertor.convert(o); return new ClickHousePayload(payload); } catch (Exception e) { + LOG.error("Failed to convert ClickHouse payload", e); return new ClickHousePayload(null); } } diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java index 902e43b..96f5b0d 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java @@ -93,10 +93,12 @@ protected void submitRequestEntries(List requestEntries, Resu try { CompletableFuture response = chClient.insert(tableName, out -> { for (ClickHousePayload requestEntry : requestEntries) { - byte[] payload = requestEntry.getPayload(); - // sum the data that is sent to ClickHouse - this.numBytesSendCounter.inc(payload.length); - out.write(payload); + if (requestEntry.getPayload() != null) { + byte[] payload = requestEntry.getPayload(); + // sum the data that is sent to ClickHouse + this.numBytesSendCounter.inc(payload.length); + out.write(payload); + } } // send the number that is sent to ClickHouse this.numRecordsSendCounter.inc(requestEntries.size()); From b7058cd4cbfc7130b66030d86a0953ea00512bd8 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Thu, 5 Jun 2025 11:33:42 +0300 Subject: [PATCH 17/23] Added logging to convertToInteger --- .../src/main/java/com/clickhouse/utils/Serialize.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java b/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java index f959994..5baa19b 100644 --- a/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java +++ b/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java @@ -85,7 +85,8 @@ public static Integer convertToInteger(Object value) { } else if (value instanceof Boolean) { return ((Boolean) value) ? 1 : 0; } else { - throw new IllegalArgumentException("Cannot convert " + value + " to Integer"); + throw new IllegalArgumentException("Cannot convert object of type " + + value.getClass().getName() + " to Integer: " + value); } } From 02e4aa7c1cbbcdbe564c6ec5c4ca774f5e58fb65 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Thu, 5 Jun 2025 14:48:12 +0300 Subject: [PATCH 18/23] Remove System.out... --- .../src/main/java/com/clickhouse/utils/Serialize.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java b/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java index 5baa19b..0781ae2 100644 --- a/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java +++ b/flink-connector-clickhouse-base/src/main/java/com/clickhouse/utils/Serialize.java @@ -114,7 +114,6 @@ public static Map mapClickHouseTypeToMethod() { // Method structure write[ClickHouse Type](OutputStream, Java type, ... ) // Date support public static void writeDate(OutputStream out, LocalDate value, boolean defaultsSupport, boolean isNullable, ClickHouseDataType dataType, boolean hasDefault, String column) throws IOException { - System.out.println("writeDate"); if (writeValuePreamble(out, defaultsSupport, isNullable, dataType, hasDefault, column, value)) { SerializerUtils.writeDate(out, value, ZoneId.of("UTC")); // TODO: check } From c17bb06c63b659020edc7474cac94190485429ac Mon Sep 17 00:00:00 2001 From: mzitnik Date: Thu, 5 Jun 2025 14:54:31 +0300 Subject: [PATCH 19/23] Remove join from logic --- .../connector/clickhouse/sink/ClickHouseAsyncWriter.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java index 96f5b0d..8222d1f 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncWriter.java @@ -102,7 +102,7 @@ protected void submitRequestEntries(List requestEntries, Resu } // send the number that is sent to ClickHouse this.numRecordsSendCounter.inc(requestEntries.size()); - LOG.info("Data that will be send to ClickHouse in bytes {} and the amount of records {}.", numBytesSendCounter.getCount(), requestEntries.size()); + LOG.info("Data that will be sent to ClickHouse in bytes {} and the amount of records {}.", numBytesSendCounter.getCount(), requestEntries.size()); out.close(); }, format, new InsertSettings().setOption(ClientConfigProperties.ASYNC_OPERATIONS.getKey(), "true")); response.whenComplete((insertResponse, throwable) -> { @@ -111,7 +111,7 @@ protected void submitRequestEntries(List requestEntries, Resu } else { handleSuccessfulRequest(resultHandler, insertResponse); } - }).join(); + }); } catch (Exception e) { LOG.error("Error: ", e); } From a77846ece5519ad9332c82f8e7d08fc6a62fd5c3 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Thu, 5 Jun 2025 15:01:10 +0300 Subject: [PATCH 20/23] Added some enhancements to ClickHouseAsyncSinkSerializer --- .../sink/ClickHouseAsyncSinkSerializer.java | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSinkSerializer.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSinkSerializer.java index be54c65..423a6b3 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSinkSerializer.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSinkSerializer.java @@ -11,23 +11,31 @@ public class ClickHouseAsyncSinkSerializer extends AsyncSinkWriterStateSerializer { private static final Logger LOG = LoggerFactory.getLogger(ClickHouseAsyncSinkSerializer.class); - + private static final int V1 = 1; @Override protected void serializeRequestToStream(ClickHousePayload clickHousePayload, DataOutputStream dataOutputStream) throws IOException { byte[] bytes = clickHousePayload.getPayload(); - dataOutputStream.writeInt(bytes.length); - dataOutputStream.write(bytes); + if (bytes != null) { + dataOutputStream.writeInt(bytes.length); + dataOutputStream.write(bytes); + } else { + dataOutputStream.writeInt(-1); + } + } private ClickHousePayload deserializeV1(DataInputStream dataInputStream) throws IOException { int len = dataInputStream.readInt(); + if (len == -1) { + return new ClickHousePayload(null); + } byte[] payload = dataInputStream.readNBytes(len); return new ClickHousePayload(payload); } @Override protected ClickHousePayload deserializeRequestFromStream(long version, DataInputStream dataInputStream) throws IOException { - if (version == 1) { + if (version == V1) { return deserializeV1(dataInputStream); } else { throw new IOException("Unsupported version: " + version); @@ -36,6 +44,6 @@ protected ClickHousePayload deserializeRequestFromStream(long version, DataInput @Override public int getVersion() { - return 1; + return V1; } } From bea0ec66851ddc728576ccf1e5d109818c4a25d0 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Thu, 5 Jun 2025 15:47:40 +0300 Subject: [PATCH 21/23] Address comments --- .../connector/clickhouse/convertor/ClickHouseConvertor.java | 1 + .../connector/clickhouse/exception/RetriableException.java | 2 ++ .../flink/connector/clickhouse/sink/ClickHouseAsyncSink.java | 3 ++- .../connector/clickhouse/sink/ClickHouseClientConfig.java | 1 + .../flink/connector/clickhouse/sink/ClickHouseSinkTests.java | 2 -- 5 files changed, 6 insertions(+), 3 deletions(-) diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java index 822ea54..d135f5f 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/convertor/ClickHouseConvertor.java @@ -12,6 +12,7 @@ public class ClickHouseConvertor implements ElementConverter { private static final Logger LOG = LoggerFactory.getLogger(ClickHouseConvertor.class); + private static final long serialVersionUID = 1L; POJOConvertor pojoConvertor = null; enum Types { diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/exception/RetriableException.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/exception/RetriableException.java index 4a85e3e..a31f7ee 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/exception/RetriableException.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/exception/RetriableException.java @@ -1,6 +1,8 @@ package org.apache.flink.connector.clickhouse.exception; public class RetriableException extends FlinkWriteException { + private static final long serialVersionUID = 1L; + public RetriableException(String message) { super(message); } diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSink.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSink.java index 54f30bc..52c9f2e 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSink.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseAsyncSink.java @@ -16,6 +16,7 @@ import java.io.IOException; import java.util.Collection; import java.util.Collections; +import java.util.Objects; public class ClickHouseAsyncSink extends AsyncSinkBase { private static final Logger LOG = LoggerFactory.getLogger(ClickHouseAsyncSink.class); @@ -41,7 +42,7 @@ public ClickHouseAsyncSink( maxTimeInBufferMS, maxRecordSizeInByte); - this.clickHouseClientConfig = clickHouseClientConfig; + this.clickHouseClientConfig = Objects.requireNonNull(clickHouseClientConfig, "ClickHouse config cannot be null");; } public void setClickHouseFormat(ClickHouseFormat clickHouseFormat) { diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java index 8e1f490..3dfec7c 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/sink/ClickHouseClientConfig.java @@ -9,6 +9,7 @@ public class ClickHouseClientConfig implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(ClickHouseClientConfig.class); + private static final long serialVersionUID = 1L; private final String url; private final String username; diff --git a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java index 3556f08..6442c5e 100644 --- a/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java +++ b/flink-connector-clickhouse-base/src/test/java/org/apache/flink/connector/clickhouse/sink/ClickHouseSinkTests.java @@ -226,8 +226,6 @@ void SimplePOJODataTest() throws Exception { clickHouseClientConfig ); - simplePOJOSink.setClickHouseFormat(ClickHouseFormat.RowBinary); - List simplePOJOList = new ArrayList<>(); for (int i = 0; i < EXPECTED_ROWS; i++) { simplePOJOList.add(new SimplePOJO(i)); From 35202ace54afcbf2be4ff2be5f6efeba9b24bc35 Mon Sep 17 00:00:00 2001 From: mzitnik Date: Thu, 5 Jun 2025 17:33:04 +0300 Subject: [PATCH 22/23] Added serialVersionUID to ClickHousePayload --- .../flink/connector/clickhouse/data/ClickHousePayload.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/data/ClickHousePayload.java b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/data/ClickHousePayload.java index d3305f4..25fb0ae 100644 --- a/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/data/ClickHousePayload.java +++ b/flink-connector-clickhouse-base/src/main/java/org/apache/flink/connector/clickhouse/data/ClickHousePayload.java @@ -7,6 +7,7 @@ public class ClickHousePayload implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(ClickHousePayload.class); + private static final long serialVersionUID = 1L; private final byte[] payload; public ClickHousePayload(byte[] payload) { From 1716d05a1aa611bca9299f2827cbaa4600d5831e Mon Sep 17 00:00:00 2001 From: mzitnik Date: Thu, 5 Jun 2025 18:02:47 +0300 Subject: [PATCH 23/23] Fixing version nammeing in github action --- .github/workflows/tests-scala.yaml | 2 +- .github/workflows/tests.yaml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/tests-scala.yaml b/.github/workflows/tests-scala.yaml index 4005630..94d65e7 100644 --- a/.github/workflows/tests-scala.yaml +++ b/.github/workflows/tests-scala.yaml @@ -22,7 +22,7 @@ jobs: shell: bash - uses: actions/checkout@v3 if: env.SKIP_STEP != 'true' - - name: Set up JDK 17 + - name: Set up JDK 21 if: env.SKIP_STEP != 'true' uses: actions/setup-java@v3 with: diff --git a/.github/workflows/tests.yaml b/.github/workflows/tests.yaml index 4ff495e..be26ff4 100644 --- a/.github/workflows/tests.yaml +++ b/.github/workflows/tests.yaml @@ -22,7 +22,7 @@ jobs: shell: bash - uses: actions/checkout@v3 if: env.SKIP_STEP != 'true' - - name: Set up JDK 17 + - name: Set up JDK 21 if: env.SKIP_STEP != 'true' uses: actions/setup-java@v3 with: