From 0d24b902e3e757ec151f6c9ddc55b68b1952392e Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Thu, 21 Jul 2022 19:27:00 -0400 Subject: [PATCH 01/13] Refactor logic to buffer records and flush. --- deploy/docker/docker-compose.yaml | 4 +- .../sink-connector-setup-schema-registry.sh | 2 +- pom.xml | 2 +- ...lickHouseSinkConnectorConfigVariables.java | 2 +- .../sink/connector/ClickHouseSinkTask.java | 4 +- .../sink/connector/db/DbWriter.java | 9 +- .../executor/ClickHouseBatchRunnable.java | 259 ++++++++++++------ .../sink/connector/model/DBCredentials.java | 28 ++ .../executor/ClickHouseBatchRunnableTest.java | 89 ++++++ 9 files changed, 305 insertions(+), 94 deletions(-) create mode 100644 src/main/java/com/altinity/clickhouse/sink/connector/model/DBCredentials.java create mode 100644 src/test/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnableTest.java diff --git a/deploy/docker/docker-compose.yaml b/deploy/docker/docker-compose.yaml index 1a9254efc..e07c02710 100644 --- a/deploy/docker/docker-compose.yaml +++ b/deploy/docker/docker-compose.yaml @@ -79,7 +79,7 @@ services: schemaregistry: container_name: schemaregistry - image: apicurio/apicurio-registry-mem:2.0.0.Final + image: apicurio/apicurio-registry-mem:latest-release #image: confluentinc/cp-schema-registry:latest restart: "no" ports: @@ -97,7 +97,7 @@ services: debezium: container_name: debezium - image: debezium/connect:1.9.2.Final + image: debezium/connect:2.0 restart: "no" ports: - "8083:8083" diff --git a/deploy/sink-connector-setup-schema-registry.sh b/deploy/sink-connector-setup-schema-registry.sh index 87910b752..378f1b7cf 100755 --- a/deploy/sink-connector-setup-schema-registry.sh +++ b/deploy/sink-connector-setup-schema-registry.sh @@ -22,7 +22,7 @@ TOPICS_TABLE_MAP="SERVER5432.test.employees_predated:employees, SERVER5432.test. #"topics.regex": "SERVER5432.sbtest.(.*), SERVER5432.test.(.*)", -"topics": "${TOPICS}", +#"topics": "${TOPICS}", cat < io.debezium debezium-core - 1.8.1.Final + 1.9.5.Final diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/ClickHouseSinkConnectorConfigVariables.java b/src/main/java/com/altinity/clickhouse/sink/connector/ClickHouseSinkConnectorConfigVariables.java index 9c0f2cbd9..f3035486e 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/ClickHouseSinkConnectorConfigVariables.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/ClickHouseSinkConnectorConfigVariables.java @@ -41,7 +41,7 @@ public class ClickHouseSinkConnectorConfigVariables { public static final String BUFFER_MAX_RECORDS = "buffer.max.records"; // Flush timeout(in milliseconds) if max records is not reached. - public static final String BUFFER_FLUSH_TIMEOUT = "buffer.flush.timeout"; + public static final String BUFFER_FLUSH_TIMEOUT = "buffer.flush.timeout.ms"; // Flag to enable prometheus metrics and to start a prometheus scrape server endpoint. public static final String ENABLE_METRICS = "metrics.enable"; diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/ClickHouseSinkTask.java b/src/main/java/com/altinity/clickhouse/sink/connector/ClickHouseSinkTask.java index 05cd734ee..6e12dc514 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/ClickHouseSinkTask.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/ClickHouseSinkTask.java @@ -123,7 +123,9 @@ private void appendToRecords(String topicName, ClickHouseStruct chs) { structs = new ConcurrentLinkedQueue<>(); } structs.add(chs); - this.records.put(topicName, structs); + synchronized (this.records) { + this.records.put(topicName, structs); + } } /** diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java b/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java index 353574236..d25e86538 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java @@ -199,7 +199,8 @@ private void updatePartitionOffsetMap(Map offsetToPartitio /** * Function to group the Query with records. - * + * Also this slices a chunk of records for processing + * from the shared data structure(ConcurrentLinkedQueue) * @param records * @return */ @@ -211,7 +212,7 @@ public Map groupQueryWithRecords(ConcurrentLinkedQueue> partitionToOffsetMap = new HashMap>(); if (records.isEmpty()) { - log.info("No Records to process"); + log.debug("No Records to process"); return partitionToOffsetMap; } @@ -343,7 +344,7 @@ public Map insert(ConcurrentLinkedQueue // HashMap> partitionToOffsetMap = new HashMap>(); if (records.isEmpty()) { - log.info("No Records to process"); + log.debug("No Records to process"); // bmd.setPartitionToOffsetMap(partitionToOffsetMap); return partitionToOffsetMap; } @@ -413,7 +414,7 @@ public void addToPreparedStatementBatch(Map topic2TableMap; - private Map>, List> queryToRecordsMap; + //private Map>, List> queryToRecordsMap; private long lastFlushTimeInMs = 0; - private Map topicToWriterMap; + /** + * Data structures with state + */ + // Map of topic names to table names. + private final Map topic2TableMap; + + // Map of topic name to CLickHouseConnection instance(DbWriter) + private Map topicToDbWriterMap; + + // Map of topic name to buffered records. + Map>, List>> topicToRecordsMap; + + private DBCredentials dbCredentials; public ClickHouseBatchRunnable(ConcurrentHashMap> records, ClickHouseSinkConnectorConfig config, Map topic2TableMap) { this.records = records; this.config = config; - if(topic2TableMap == null) { + if (topic2TableMap == null) { this.topic2TableMap = new HashMap(); } else { this.topic2TableMap = topic2TableMap; } - this.queryToRecordsMap = new HashMap<>(); - this.topicToWriterMap = new HashMap<>(); + //this.queryToRecordsMap = new HashMap<>(); + this.topicToDbWriterMap = new HashMap<>(); + this.topicToRecordsMap = new HashMap<>(); + + this.dbCredentials = parseDBConfiguration(); + + } + + private DBCredentials parseDBConfiguration() { + DBCredentials dbCredentials = new DBCredentials(); + + dbCredentials.setHostName(config.getString(ClickHouseSinkConnectorConfigVariables.CLICKHOUSE_URL)); + dbCredentials.setDatabase(config.getString(ClickHouseSinkConnectorConfigVariables.CLICKHOUSE_DATABASE)); + dbCredentials.setPort(config.getInt(ClickHouseSinkConnectorConfigVariables.CLICKHOUSE_PORT)); + dbCredentials.setUserName(config.getString(ClickHouseSinkConnectorConfigVariables.CLICKHOUSE_USER)); + dbCredentials.setPassword(config.getString(ClickHouseSinkConnectorConfigVariables.CLICKHOUSE_PASS)); + + return dbCredentials; } + /** + * Main run loop of the thread + * which is called based on the schedule + * Default: 100 msecs + */ @Override public void run() { - String dbHostName = config.getString(ClickHouseSinkConnectorConfigVariables.CLICKHOUSE_URL); - String database = config.getString(ClickHouseSinkConnectorConfigVariables.CLICKHOUSE_DATABASE); - Integer port = config.getInt(ClickHouseSinkConnectorConfigVariables.CLICKHOUSE_PORT); - String userName = config.getString(ClickHouseSinkConnectorConfigVariables.CLICKHOUSE_USER); - String password = config.getString(ClickHouseSinkConnectorConfigVariables.CLICKHOUSE_PASS); - //String tableName = config.getString(ClickHouseSinkConnectorConfigVariables.CLICKHOUSE_TABLE); - + Long taskId = config.getLong(ClickHouseSinkConnectorConfigVariables.TASK_ID); + int numRecords = records.size(); + if (numRecords <= 0) { + log.debug(String.format("No records to process ThreadId(%s), TaskId(%s)", Thread.currentThread().getId(), taskId)); + return; + } // Topic Name -> List of records for (Map.Entry> entry : this.records.entrySet()) { + if (entry.getValue().size() > 0) { + processRecordsByTopic(entry.getKey(), entry.getValue()); + } + } + } - String topicName = entry.getKey(); - DbWriter writer = null; + /** + * Function to retrieve table name from topic name + * + * @param topicName + * @return Table Name + */ + public String getTableFromTopic(String topicName) { + String tableName = null; + + if (this.topic2TableMap.containsKey(topicName) == false) { + tableName = Utils.getTableNameFromTopic(topicName); + this.topic2TableMap.put(topicName, tableName); + } else { + tableName = this.topic2TableMap.get(topicName); + } - //The user parameter will override the topic mapping to table. - String tableName; + return tableName; + } - if (this.topic2TableMap.containsKey(topicName) == false) { - tableName = Utils.getTableNameFromTopic(topicName); - this.topic2TableMap.put(topicName, tableName); - } else { - tableName = this.topic2TableMap.get(topicName); - } + public DbWriter getDbWriterForTable(String topicName, String tableName, ClickHouseStruct record) { + DbWriter writer = null; - if (entry.getValue().size() > 0) { - UUID blockUuid = UUID.randomUUID(); - - Long taskId = config.getLong(ClickHouseSinkConnectorConfigVariables.TASK_ID); - int numRecords = entry.getValue().size(); - log.debug("*************** BULK INSERT TO CLICKHOUSE - RECORDS:" + numRecords + "************** task(" + taskId + ")" + " Thread ID: " + Thread.currentThread().getName()); - - // Initialize Timer to track time taken to transform and insert to Clickhouse. - Timer timer = Metrics.timer("Bulk Insert: " + blockUuid + " Size:" + entry.getValue().size()); - Timer.Context context = timer.time(); - - // Check if DB instance exists for the current topic - // or else create a new one. - if(this.topicToWriterMap.containsKey(topicName)) { - writer = this.topicToWriterMap.get(topicName); - } else { - writer = new DbWriter(dbHostName, port, database, tableName, userName, password, this.config, entry.getValue().peek()); - this.topicToWriterMap.put(topicName, writer); - } - Map partitionToOffsetMap; - synchronized (this.records) { - - partitionToOffsetMap = writer.insert(entry.getValue(), queryToRecordsMap); - - long currentTime = System.currentTimeMillis(); - long diffInMs = currentTime - lastFlushTimeInMs; - long bufferFlushTimeout = this.config.getLong(ClickHouseSinkConnectorConfigVariables.BUFFER_FLUSH_TIMEOUT); - - if(diffInMs > bufferFlushTimeout) { - // Time to flush. - log.info("**** TIME EXCEEDED %s to FLUSH", bufferFlushTimeout); - writer.addToPreparedStatementBatch(queryToRecordsMap); - lastFlushTimeInMs = currentTime; - } else { - long totalSize = 0; - for (Map.Entry>, List> mutablePairListEntry : queryToRecordsMap.entrySet()) { - totalSize += mutablePairListEntry.getValue().size(); - } - long minRecordsToFlush = config.getLong(ClickHouseSinkConnectorConfigVariables.BUFFER_MAX_RECORDS); - - if(totalSize >= minRecordsToFlush) { - log.info("**** MAX RECORDS EXCEEDED to FLUSH:" + "Total Records: " + totalSize); - writer.addToPreparedStatementBatch(queryToRecordsMap); - lastFlushTimeInMs = currentTime; - } - - } - } - if(this.config.getBoolean(ClickHouseSinkConnectorConfigVariables.ENABLE_KAFKA_OFFSET)) { - log.info("***** KAFKA OFFSET MANAGEMENT ENABLED *****"); - DbKafkaOffsetWriter dbKafkaOffsetWriter = new DbKafkaOffsetWriter(dbHostName, port, database, "topic_offset_metadata", userName, password, this.config); - try { - dbKafkaOffsetWriter.insertTopicOffsetMetadata(partitionToOffsetMap); - } catch (SQLException e) { - log.error("Error persisting offsets to CH", e); - } - } - //context.stop(); + + // Check if DB instance exists for the current topic + // or else create a new one. + if (this.topicToDbWriterMap.containsKey(topicName)) { + writer = this.topicToDbWriterMap.get(topicName); + } else { + writer = new DbWriter(this.dbCredentials.getHostName(), this.dbCredentials.getPort(), + this.dbCredentials.getDatabase(), tableName, this.dbCredentials.getUserName(), + this.dbCredentials.getPassword(), this.config, record); + this.topicToDbWriterMap.put(topicName, writer); + } + + return writer; + } +// +// UUID blockUuid = UUID.randomUUID(); +// +// // Initialize Timer to track time taken to transform and insert to Clickhouse. +// Timer timer = Metrics.timer("Bulk Insert: " + blockUuid + " Size:" + numRecords); +// Timer.Context context = timer.time(); +// +// Map partitionToOffsetMap; + + /** + * Function to process records + * + * @param topicName + * @param records + */ + private void processRecordsByTopic(String topicName, ConcurrentLinkedQueue records) { + + //The user parameter will override the topic mapping to table. + String tableName = getTableFromTopic(topicName); + DbWriter writer = getDbWriterForTable(topicName, tableName, records.peek()); + + // Step 1: The Batch Insert with preparedStatement in JDBC + // works by forming the Query and then adding records to the Batch. + // This step creates a Map of Query -> Records(List of ClickHouseStruct) + Map>, List> queryToRecordsMap; + + if(topicToRecordsMap.containsKey(topicName)) { + queryToRecordsMap = topicToRecordsMap.get(topicName); + } else { + queryToRecordsMap = new HashMap<>(); + topicToRecordsMap.put(topicName, queryToRecordsMap); + } + + Map partitionToOffsetMap = writer.groupQueryWithRecords(records, queryToRecordsMap); + + if(flushRecordsToClickHouse(writer, queryToRecordsMap)) { + // Remove the entry. + queryToRecordsMap.remove(topicName); + } + + if (this.config.getBoolean(ClickHouseSinkConnectorConfigVariables.ENABLE_KAFKA_OFFSET)) { + log.info("***** KAFKA OFFSET MANAGEMENT ENABLED *****"); + DbKafkaOffsetWriter dbKafkaOffsetWriter = new DbKafkaOffsetWriter(dbCredentials.getHostName(), dbCredentials.getPort(), dbCredentials.getDatabase(), + "topic_offset_metadata", dbCredentials.getUserName(), dbCredentials.getPassword(), this.config); + try { + dbKafkaOffsetWriter.insertTopicOffsetMetadata(partitionToOffsetMap); + } catch (SQLException e) { + log.error("Error persisting offsets to CH", e); + } + } + //context.stop(); // Metrics.updateSinkRecordsCounter(blockUuid.toString(), taskId, topicName, tableName, // bmd.getPartitionToOffsetMap(), numRecords, bmd.getMinSourceLag(), // bmd.getMaxSourceLag(), bmd.getMinConsumerLag(), bmd.getMaxConsumerLag()); + } + + /** + * Function that flushes records to ClickHouse if + * there are minimum records or if the flush timeout has reached. + * @param writer + * @param queryToRecordsMap + * @return + */ + private boolean flushRecordsToClickHouse(DbWriter writer, Map>, List> queryToRecordsMap) { + + boolean result = false; + + long currentTime = System.currentTimeMillis(); + long diffInMs = currentTime - lastFlushTimeInMs; + long bufferFlushTimeout = this.config.getLong(ClickHouseSinkConnectorConfigVariables.BUFFER_FLUSH_TIMEOUT); + + // Step 2: Check if the buffer can be flushed + // One if the max buffer size is reached + // or if the Buffer flush timeout is reached. + if (diffInMs > bufferFlushTimeout) { + // Time to flush. + log.info(String.format("*** TIME EXCEEDED %s to FLUSH", bufferFlushTimeout)); + writer.addToPreparedStatementBatch(queryToRecordsMap); + lastFlushTimeInMs = currentTime; + result = true; + } else { + long totalSize = 0; + for (Map.Entry>, List> + mutablePairListEntry : queryToRecordsMap.entrySet()) { + totalSize += mutablePairListEntry.getValue().size(); + } + long minRecordsToFlush = config.getLong(ClickHouseSinkConnectorConfigVariables.BUFFER_MAX_RECORDS); + + if (totalSize >= minRecordsToFlush) { + log.info("**** MAX RECORDS EXCEEDED to FLUSH:" + "Total Records: " + totalSize); + writer.addToPreparedStatementBatch(queryToRecordsMap); + lastFlushTimeInMs = currentTime; + result = true; } } + + return result; } } diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/model/DBCredentials.java b/src/main/java/com/altinity/clickhouse/sink/connector/model/DBCredentials.java new file mode 100644 index 000000000..e251fc549 --- /dev/null +++ b/src/main/java/com/altinity/clickhouse/sink/connector/model/DBCredentials.java @@ -0,0 +1,28 @@ +package com.altinity.clickhouse.sink.connector.model; + +import lombok.Getter; +import lombok.Setter; + +public class DBCredentials { + + @Getter + @Setter + private String hostName; + + @Getter + @Setter + private String database; + + @Getter + @Setter + private Integer port; + + @Getter + @Setter + private String userName; + + @Getter + @Setter + private String password; + +} diff --git a/src/test/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnableTest.java b/src/test/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnableTest.java new file mode 100644 index 000000000..5c34bf6fe --- /dev/null +++ b/src/test/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnableTest.java @@ -0,0 +1,89 @@ +package com.altinity.clickhouse.sink.connector.executor; + +import com.altinity.clickhouse.sink.connector.ClickHouseSinkConnectorConfig; +import com.altinity.clickhouse.sink.connector.converters.ClickHouseConverter; +import com.altinity.clickhouse.sink.connector.model.ClickHouseStruct; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; + +public class ClickHouseBatchRunnableTest { + + + ConcurrentHashMap> + records = new ConcurrentHashMap<>(); + Map topic2TableMap = new HashMap<>(); + + @Before + public void initTest() { + + + ClickHouseStruct ch1 = new ClickHouseStruct(10, "SERVER5432.test.customers", getKafkaStruct(), 2, System.currentTimeMillis(), null, getKafkaStruct(), null, ClickHouseConverter.CDC_OPERATION.CREATE); + ClickHouseStruct ch2 = new ClickHouseStruct(8, "SERVER5432.test.customers", getKafkaStruct(), 2, System.currentTimeMillis() ,null, getKafkaStruct(), null, ClickHouseConverter.CDC_OPERATION.CREATE); + ClickHouseStruct ch3 = new ClickHouseStruct(1000, "SERVER5432.test.customers", getKafkaStruct(), 2, System.currentTimeMillis(), null, getKafkaStruct(), null, ClickHouseConverter.CDC_OPERATION.CREATE); + + + ClickHouseStruct ch4 = new ClickHouseStruct(1020, "SERVER5432.test.products", getKafkaStruct(), 3, System.currentTimeMillis(), null, getKafkaStruct(), null, ClickHouseConverter.CDC_OPERATION.CREATE); + ClickHouseStruct ch5 = new ClickHouseStruct(1400, "SERVER5432.test.products", getKafkaStruct(), 2, System.currentTimeMillis(), null, getKafkaStruct(), null, ClickHouseConverter.CDC_OPERATION.CREATE); + ClickHouseStruct ch6 = new ClickHouseStruct(1010, "SERVER5432.test.products", getKafkaStruct(), 2, System.currentTimeMillis(), null, getKafkaStruct(), null, ClickHouseConverter.CDC_OPERATION.CREATE); + + ConcurrentLinkedQueue customersQueue = new ConcurrentLinkedQueue<>(); + customersQueue.add(ch1); + customersQueue.add(ch2); + customersQueue.add(ch3); + + ConcurrentLinkedQueue productsQueue = new ConcurrentLinkedQueue<>(); + productsQueue.add(ch1); + productsQueue.add(ch2); + productsQueue.add(ch3); + + records.put("SERVER5432.test.customers", customersQueue); + records.put("SERVER5432.test.products", productsQueue); + + this.topic2TableMap.put("SERVER5432.test.customers", "customers"); + this.topic2TableMap.put("SERVER5432.test.products", "products"); + this.topic2TableMap.put("SERVER5432.test.employees", "employees"); + + } + + + public Struct getKafkaStruct() { + Schema kafkaConnectSchema = SchemaBuilder + .struct() + .field("first_name", Schema.STRING_SCHEMA) + .field("last_name", Schema.STRING_SCHEMA) + .field("quantity", Schema.INT32_SCHEMA) + .field("amount", Schema.FLOAT64_SCHEMA) + .field("employed", Schema.BOOLEAN_SCHEMA) + .build(); + + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put("first_name", "John"); + kafkaConnectStruct.put("last_name", "Doe"); + kafkaConnectStruct.put("quantity", 100); + kafkaConnectStruct.put("amount", 23.223); + kafkaConnectStruct.put("employed", true); + + + return kafkaConnectStruct; + } + + @Test + public void testGetTableNameFromTopic() { + ClickHouseSinkConnectorConfig config = new ClickHouseSinkConnectorConfig(new HashMap()); + ClickHouseBatchRunnable run = new ClickHouseBatchRunnable(this.records, config, this.topic2TableMap); + + String tableName = run.getTableFromTopic("SERVER5432.test.customers"); + + Assert.assertTrue(tableName.equalsIgnoreCase("customers")); + + } +} From 2839b960c229d8b2618f44a5d2f0ebac16625414 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Fri, 22 Jul 2022 12:18:42 -0400 Subject: [PATCH 02/13] Combine all the sysbench tests to a single bash script. --- .../sink-connector-setup-schema-registry.sh | 2 +- deploy/sysbench/run_sysbench_bulk_insert.sh | 54 ----------- .../sysbench/run_sysbench_insert_load_test.sh | 25 ------ deploy/sysbench/run_sysbench_oltp_delete.sh | 55 ------------ .../run_sysbench_read_write_load_test.sh | 19 ---- deploy/sysbench/run_sysbench_tests.sh | 89 +++++++++++++++++++ deploy/sysbench/run_sysbench_update_index.sh | 55 ------------ .../sysbench/run_sysbench_update_non_index.sh | 55 ------------ ...lickHouseSinkConnectorConfigVariables.java | 4 +- 9 files changed, 92 insertions(+), 266 deletions(-) delete mode 100755 deploy/sysbench/run_sysbench_bulk_insert.sh delete mode 100755 deploy/sysbench/run_sysbench_insert_load_test.sh delete mode 100755 deploy/sysbench/run_sysbench_oltp_delete.sh delete mode 100755 deploy/sysbench/run_sysbench_read_write_load_test.sh create mode 100755 deploy/sysbench/run_sysbench_tests.sh delete mode 100755 deploy/sysbench/run_sysbench_update_index.sh delete mode 100755 deploy/sysbench/run_sysbench_update_non_index.sh diff --git a/deploy/sink-connector-setup-schema-registry.sh b/deploy/sink-connector-setup-schema-registry.sh index 378f1b7cf..0755ca474 100755 --- a/deploy/sink-connector-setup-schema-registry.sh +++ b/deploy/sink-connector-setup-schema-registry.sh @@ -56,7 +56,7 @@ cat <, test_name should be one of the following + bulk_insert, oltp_insert, oltp_update_index, oltp_update_index, oltp_update_non_index" +} + +### Supported Sysbench tests +#declare -a supported_test_names=("bulk_insert" "oltp_insert", "oltp_update_index", "oltp_delete", "oltp_update_index", "oltp_update_non_index") +supported_test_names=() +supported_test_names+=('bulk_insert') +supported_test_names+=('oltp_insert') +supported_test_names+=('oltp_delete') +supported_test_names+=('oltp_update_index') +supported_test_names+=('oltp_update_non_index') + +### Sysbench configuration +num_threads=16 +time=60 # IN Seconds + +mysql_host=127.0.0.1 +mysql_port=3306 +mysql_username=root +mysql_password=root +mysql_db=sbtest + +sysbench_command() { +sysbench \ +/usr/share/sysbench/${1}.lua \ +--report-interval=2 \ +--threads=${num_threads} \ +--rate=0 \ +--time=${time} \ +--db-driver=mysql \ +--mysql-host=${mysql_host} \ +--mysql-port=${mysql_port} \ +--mysql-user=${mysql_username} \ +--mysql-db=${mysql_db} \ +--mysql-password=${mysql_password} \ +--tables=1 \ +--table-size=100 \ +$2 +} + +while : +do + case "$1" in + -t | --test-name ) + test_name="$2" + if grep -q ${test_name} <<< "${supported_test_names[@]}"; then + echo "Supported test name" + echo "Running Sysbench cleanup ...." + eval sysbench_command ${test_name} "cleanup" + sleep 2 + echo "Running Sysbench prepare ...." + eval sysbench_command ${test_name} "prepare" + sleep 2 + echo "Running Sysbench run ...." + eval sysbench_command ${test_name} "run" + + else + echo "Not supported test_name" + help + fi + shift 2 + ;; + -h | --help) + help + exit 2 + ;; + --) + shift; + break + ;; + *) + echo "Unexpected option: $1" + help + exit 2 + ;; + esac +done + diff --git a/deploy/sysbench/run_sysbench_update_index.sh b/deploy/sysbench/run_sysbench_update_index.sh deleted file mode 100755 index d360e201b..000000000 --- a/deploy/sysbench/run_sysbench_update_index.sh +++ /dev/null @@ -1,55 +0,0 @@ -#!/bin/bash - -sysbench \ -/usr/share/sysbench/oltp_update_index.lua \ ---report-interval=2 \ ---threads=16 \ ---rate=0 \ ---time=0 \ ---db-driver=mysql \ ---mysql-host=127.0.0.1 \ ---mysql-port=3306 \ ---mysql-user=root \ ---mysql-db=sbtest \ ---mysql-password=root \ ---tables=1 \ ---table-size=10000 \ ---debug \ ---verbosity=5 \ -cleanup - -sysbench \ -/usr/share/sysbench/oltp_update_index.lua \ ---report-interval=2 \ ---threads=16 \ ---rate=0 \ ---time=0 \ ---db-driver=mysql \ ---mysql-host=127.0.0.1 \ ---mysql-port=3306 \ ---mysql-user=root \ ---mysql-db=sbtest \ ---mysql-password=root \ ---tables=1 \ ---table-size=10000 \ ---debug \ ---verbosity=5 \ -prepare - -sysbench \ -/usr/share/sysbench/oltp_update_index.lua \ ---report-interval=2 \ ---threads=16 \ ---rate=0 \ ---time=0 \ ---db-driver=mysql \ ---mysql-host=127.0.0.1 \ ---mysql-port=3306 \ ---mysql-user=root \ ---mysql-db=sbtest \ ---mysql-password=root \ ---tables=1 \ ---table-size=10000 \ ---debug \ ---verbosity=5 \ -run \ No newline at end of file diff --git a/deploy/sysbench/run_sysbench_update_non_index.sh b/deploy/sysbench/run_sysbench_update_non_index.sh deleted file mode 100755 index 208349678..000000000 --- a/deploy/sysbench/run_sysbench_update_non_index.sh +++ /dev/null @@ -1,55 +0,0 @@ -#!/bin/bash - -sysbench \ -/usr/share/sysbench/oltp_update_non_index.lua \ ---report-interval=2 \ ---threads=16 \ ---rate=0 \ ---time=0 \ ---db-driver=mysql \ ---mysql-host=127.0.0.1 \ ---mysql-port=3306 \ ---mysql-user=root \ ---mysql-db=sbtest \ ---mysql-password=root \ ---tables=1 \ ---table-size=10000 \ ---debug \ ---verbosity=5 \ -cleanup - -sysbench \ -/usr/share/sysbench/oltp_update_non_index.lua \ ---report-interval=2 \ ---threads=16 \ ---rate=0 \ ---time=0 \ ---db-driver=mysql \ ---mysql-host=127.0.0.1 \ ---mysql-port=3306 \ ---mysql-user=root \ ---mysql-db=sbtest \ ---mysql-password=root \ ---tables=1 \ ---table-size=10000 \ ---debug \ ---verbosity=5 \ -prepare - -sysbench \ -/usr/share/sysbench/oltp_update_non_index.lua \ ---report-interval=2 \ ---threads=16 \ ---rate=0 \ ---time=0 \ ---db-driver=mysql \ ---mysql-host=127.0.0.1 \ ---mysql-port=3306 \ ---mysql-user=root \ ---mysql-db=sbtest \ ---mysql-password=root \ ---tables=1 \ ---table-size=10000 \ ---debug \ ---verbosity=5 \ -run \ No newline at end of file diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/ClickHouseSinkConnectorConfigVariables.java b/src/main/java/com/altinity/clickhouse/sink/connector/ClickHouseSinkConnectorConfigVariables.java index f3035486e..2ebef7bea 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/ClickHouseSinkConnectorConfigVariables.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/ClickHouseSinkConnectorConfigVariables.java @@ -34,8 +34,8 @@ public class ClickHouseSinkConnectorConfigVariables { // Column to store the raw data. public static final String STORE_RAW_DATA_COLUMN = "store.raw.data.column"; - // Buffer flush time in seconds. - public static final String BUFFER_FLUSH_TIME = "buffer.flush.time"; + // Buffer flush time in milliseconds. + public static final String BUFFER_FLUSH_TIME = "buffer.flush.time.ms"; // Maximum size of buffer before its flushed. public static final String BUFFER_MAX_RECORDS = "buffer.max.records"; From 17604d1ab64514612690a65ef5eadc048fe755f1 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Fri, 22 Jul 2022 19:01:19 -0400 Subject: [PATCH 03/13] Added integration tests to compare count of seed data and sysbench tables. --- ...ebezium-connector-setup-schema-registry.sh | 4 +- deploy/debezium-connector-setup-sysbench.sh | 2 +- deploy/docker/start-docker-compose.sh | 4 +- deploy/docker/stop-docker-compose.sh | 1 + .../sink-connector-setup-schema-registry.sh | 2 +- deploy/sql/init_mysql.sql | 7 +- deploy/sysbench/run_sysbench_tests.sh | 6 +- .../executor/ClickHouseBatchRunnable.java | 51 ++++++++------- tests/clickhouse_connection.py | 5 +- tests/mysql_connection.py | 3 + tests/test_seed_data_count.py | 64 +++++++++++++++++++ 11 files changed, 113 insertions(+), 36 deletions(-) create mode 100644 tests/test_seed_data_count.py diff --git a/deploy/debezium-connector-setup-schema-registry.sh b/deploy/debezium-connector-setup-schema-registry.sh index ce9377bc9..daa427b44 100755 --- a/deploy/debezium-connector-setup-schema-registry.sh +++ b/deploy/debezium-connector-setup-schema-registry.sh @@ -97,9 +97,9 @@ cat <, test_name should be one of the following - bulk_insert, oltp_insert, oltp_update_index, oltp_update_index, oltp_update_non_index" + echo "./run_sysbench_tests.sh -t , test_name should be one of the following + bulk_insert, oltp_insert, oltp_delete, oltp_update_index, oltp_update_non_index" } ### Supported Sysbench tests @@ -76,6 +77,7 @@ do exit 2 ;; --) + help shift; break ;; diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnable.java b/src/main/java/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnable.java index a2adb5e06..050a1fe07 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnable.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnable.java @@ -213,30 +213,33 @@ private boolean flushRecordsToClickHouse(DbWriter writer, Map bufferFlushTimeout) { - // Time to flush. - log.info(String.format("*** TIME EXCEEDED %s to FLUSH", bufferFlushTimeout)); - writer.addToPreparedStatementBatch(queryToRecordsMap); - lastFlushTimeInMs = currentTime; - result = true; - } else { - long totalSize = 0; - for (Map.Entry>, List> - mutablePairListEntry : queryToRecordsMap.entrySet()) { - totalSize += mutablePairListEntry.getValue().size(); - } - long minRecordsToFlush = config.getLong(ClickHouseSinkConnectorConfigVariables.BUFFER_MAX_RECORDS); - - if (totalSize >= minRecordsToFlush) { - log.info("**** MAX RECORDS EXCEEDED to FLUSH:" + "Total Records: " + totalSize); - writer.addToPreparedStatementBatch(queryToRecordsMap); - lastFlushTimeInMs = currentTime; - result = true; - } - } + writer.addToPreparedStatementBatch(queryToRecordsMap); + result = true; +// +// // Step 2: Check if the buffer can be flushed +// // One if the max buffer size is reached +// // or if the Buffer flush timeout is reached. +// if (diffInMs > bufferFlushTimeout) { +// // Time to flush. +// log.info(String.format("*** TIME EXCEEDED %s to FLUSH", bufferFlushTimeout)); +// writer.addToPreparedStatementBatch(queryToRecordsMap); +// lastFlushTimeInMs = currentTime; +// result = true; +// } else { +// long totalSize = 0; +// for (Map.Entry>, List> +// mutablePairListEntry : queryToRecordsMap.entrySet()) { +// totalSize += mutablePairListEntry.getValue().size(); +// } +// long minRecordsToFlush = config.getLong(ClickHouseSinkConnectorConfigVariables.BUFFER_MAX_RECORDS); +// +// if (totalSize >= minRecordsToFlush) { +// log.info("**** MAX RECORDS EXCEEDED to FLUSH:" + "Total Records: " + totalSize); +// writer.addToPreparedStatementBatch(queryToRecordsMap); +// lastFlushTimeInMs = currentTime; +// result = true; +// } +// } return result; } diff --git a/tests/clickhouse_connection.py b/tests/clickhouse_connection.py index a11b73e01..e27d221d6 100644 --- a/tests/clickhouse_connection.py +++ b/tests/clickhouse_connection.py @@ -17,4 +17,7 @@ def create_connection(self): def execute_sql(self, query): result = self.client.execute(query) - return result \ No newline at end of file + return result + + def close(self): + self.client.disconnect() \ No newline at end of file diff --git a/tests/mysql_connection.py b/tests/mysql_connection.py index 3cb0fb7af..33f8e3ab5 100644 --- a/tests/mysql_connection.py +++ b/tests/mysql_connection.py @@ -50,6 +50,7 @@ def get_column_names(self, sql): def execute_sql(self, sql, data=None): + result = None if self.conn.is_connected(): self.cursor = self.conn.cursor() @@ -68,6 +69,8 @@ def execute_sql(self, sql, data=None): except Exception as e: print("Error executing SQL", e) + return result + def get_connection(self) -> MySQLConnection: return self.conn diff --git a/tests/test_seed_data_count.py b/tests/test_seed_data_count.py new file mode 100644 index 000000000..0f92003f9 --- /dev/null +++ b/tests/test_seed_data_count.py @@ -0,0 +1,64 @@ +import os +import time +import unittest +from datetime import date + +from tests.clickhouse_connection import ClickHouseConnection +from tests.mysql_connection import MySqlConnection +from fake_data import FakeData + +class MyTestCase(unittest.TestCase): + + conn = None + + @classmethod + def setUpClass(cls): + print("Setup class") + + + @classmethod + def tearDownClass(cls): + print("Teardown class") + + + def test_seed_data_count(self): + mysql_conn = MySqlConnection() + mysql_conn.create_connection() + + clickhouse_conn = ClickHouseConnection('localhost', 'root', 'root', 'test') + clickhouse_conn.create_connection() + + ch_employees_count = clickhouse_conn.execute_sql("select count(*) from employees") + mysql_employees_count = mysql_conn.execute_sql("select count(*) from employees_predated") + + ch_customers_count = clickhouse_conn.execute_sql("select count(*) from customers") + mysql_customers_count = mysql_conn.execute_sql("select count(*) from customers") + + ch_products_count = clickhouse_conn.execute_sql("select count(*) from products") + mysql_products_count = mysql_conn.execute_sql("select count(*) from products") + + mysql_conn.close() + clickhouse_conn.close() + + self.assertEqual(ch_employees_count[0], mysql_employees_count) + self.assertEqual(ch_customers_count[0], mysql_customers_count) + self.assertEqual(ch_products_count[0], mysql_products_count) + + + def test_sysbench_test_data_count(self): + + os.environ['DB_NAME'] = 'sbtest' + mysql_conn = MySqlConnection() + mysql_conn.create_connection() + + clickhouse_conn = ClickHouseConnection('localhost', 'root', 'root', 'test') + clickhouse_conn.create_connection() + + clickhouse_conn.execute_sql('optimize table sbtest1 final') + ch_count = clickhouse_conn.execute_sql("select count(*) from sbtest1") + mysql_count = mysql_conn.execute_sql("select count(*) from sbtest1") + + mysql_conn.close() + clickhouse_conn.close() + + self.assertEqual(ch_count[0], mysql_count) From 6ce8c4349d85a408c98aa67e371e2fac9afb1d4f Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Mon, 25 Jul 2022 15:41:47 -0400 Subject: [PATCH 04/13] Added steps to create grafana docker image for debezium connector and add dashboard, datasources. --- deploy/config/grafana/README.md | 3 + .../grafana/dashboards/debezium/dashboard.yml | 10 + .../dashboards/debezium/datasource.yml | 8 + .../debezium/debezium-dashboard.json | 1364 ++++++++ .../debezium/debezium-mysql-connector.json | 3100 +++++++++++++++++ deploy/config/prometheus.yml | 10 +- deploy/docker/docker-compose.yaml | 29 +- deploy/sink-connector-setup-sysbench.sh | 77 + deploy/sysbench/compare_mysql_ch.sh | 2 +- docker/debezium_jmx/Dockerfile | 9 + docker/debezium_jmx/config.yml | 24 + docker/grafana/Dockerfile | 6 + docker/grafana/config/Datasource.yml | 10 + docker/grafana/config/dashboard.yml | 10 + docker/grafana/config/datasource.yml | 8 + docker/grafana/config/debezium-dashboard.json | 1364 ++++++++ .../executor/ClickHouseBatchRunnable.java | 10 +- .../connector/model/ClickHouseStruct.java | 10 - 18 files changed, 6031 insertions(+), 23 deletions(-) create mode 100644 deploy/config/grafana/README.md create mode 100644 deploy/config/grafana/dashboards/debezium/dashboard.yml create mode 100644 deploy/config/grafana/dashboards/debezium/datasource.yml create mode 100644 deploy/config/grafana/dashboards/debezium/debezium-dashboard.json create mode 100644 deploy/config/grafana/dashboards/debezium/debezium-mysql-connector.json create mode 100755 deploy/sink-connector-setup-sysbench.sh create mode 100644 docker/debezium_jmx/Dockerfile create mode 100644 docker/debezium_jmx/config.yml create mode 100644 docker/grafana/Dockerfile create mode 100644 docker/grafana/config/Datasource.yml create mode 100644 docker/grafana/config/dashboard.yml create mode 100644 docker/grafana/config/datasource.yml create mode 100644 docker/grafana/config/debezium-dashboard.json diff --git a/deploy/config/grafana/README.md b/deploy/config/grafana/README.md new file mode 100644 index 000000000..adb25defe --- /dev/null +++ b/deploy/config/grafana/README.md @@ -0,0 +1,3 @@ +### Debezium Monitoring + +https://github.com/debezium/debezium-examples/tree/main/monitoring diff --git a/deploy/config/grafana/dashboards/debezium/dashboard.yml b/deploy/config/grafana/dashboards/debezium/dashboard.yml new file mode 100644 index 000000000..8673131f8 --- /dev/null +++ b/deploy/config/grafana/dashboards/debezium/dashboard.yml @@ -0,0 +1,10 @@ +apiVersion: 1 +providers: + - name: 'default' + orgId: 1 + folder: '' + type: file + disableDeletion: false + updateIntervalSeconds: 10 #how often Grafana will scan for changed dashboards + options: + path: /var/lib/grafana/dashboards \ No newline at end of file diff --git a/deploy/config/grafana/dashboards/debezium/datasource.yml b/deploy/config/grafana/dashboards/debezium/datasource.yml new file mode 100644 index 000000000..ba665915f --- /dev/null +++ b/deploy/config/grafana/dashboards/debezium/datasource.yml @@ -0,0 +1,8 @@ +apiVersion: 1 + +datasources: + - name: prometheus + type: prometheus + url: http://prometheus:9090 + access: proxy + version: 1 \ No newline at end of file diff --git a/deploy/config/grafana/dashboards/debezium/debezium-dashboard.json b/deploy/config/grafana/dashboards/debezium/debezium-dashboard.json new file mode 100644 index 000000000..67491a4a2 --- /dev/null +++ b/deploy/config/grafana/dashboards/debezium/debezium-dashboard.json @@ -0,0 +1,1364 @@ +{ + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": "-- Grafana --", + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "type": "dashboard" + } + ] + }, + "editable": true, + "gnetId": null, + "graphTooltip": 0, + "id": 1, + "iteration": 1544085491414, + "links": [], + "panels": [ + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 0 + }, + "id": 12, + "panels": [], + "title": "Streaming", + "type": "row" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorPostfix": false, + "colorPrefix": false, + "colorValue": true, + "colors": [ + "#299c46", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "prometheus", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": true, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 5, + "w": 3, + "x": 0, + "y": 1 + }, + "id": 14, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_MilliSecondsSinceLastEvent{plugin=\"$connector_type\",name=\"$connector_name\",context=~\"(binlog|streaming)\"}", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": "10000,60000", + "title": "Time since last event", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "current" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "prometheus", + "fill": 1, + "gridPos": { + "h": 9, + "w": 12, + "x": 3, + "y": 1 + }, + "id": 16, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_TotalNumberOfEventsSeen{plugin=\"$connector_type\",name=\"$connector_name\",context=~\"(binlog|streaming)\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Total events received", + "refId": "A" + }, + { + "expr": "debezium_metrics_NumberOfEventsSkipped{plugin=\"$connector_type\",name=\"$connector_name\",context=~\"(binlog|streaming)\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Events skipped", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Event count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#d44a3a", + "rgba(237, 129, 40, 0.89)", + "#299c46" + ], + "datasource": "prometheus", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 3, + "x": 0, + "y": 6 + }, + "id": 18, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_Connected{plugin=\"$connector_type\",name=\"$connector_name\",context=~\"(binlog|streaming)\"}", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": "0,1", + "title": "Connected", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + }, + { + "op": "=", + "text": "No", + "value": "0" + }, + { + "op": "=", + "text": "Yes", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 10 + }, + "id": 10, + "panels": [], + "title": "Snapshot", + "type": "row" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorPostfix": false, + "colorPrefix": false, + "colorValue": false, + "colors": [ + "#299c46", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "prometheus", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 2, + "w": 3, + "x": 0, + "y": 11 + }, + "id": 25, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_TotalTableCount{plugin=\"$connector_type\",name=\"$connector_name\",context=\"snapshot\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Total", + "refId": "A" + } + ], + "thresholds": "", + "title": "Table count", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorPostfix": false, + "colorPrefix": false, + "colorValue": false, + "colors": [ + "#299c46", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "prometheus", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 2, + "w": 3, + "x": 3, + "y": 11 + }, + "id": 28, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_RemainingTableCount{plugin=\"$connector_type\",name=\"$connector_name\",context=\"snapshot\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Remaining", + "refId": "B" + } + ], + "thresholds": "", + "title": "Remaining tables", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": false, + "colors": [ + "#d44a3a", + "rgba(237, 129, 40, 0.89)", + "#299c46" + ], + "datasource": "prometheus", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 2, + "w": 3, + "x": 6, + "y": 11 + }, + "id": 27, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_SnapshotRunning{plugin=\"$connector_type\",name=\"$connector_name\",context=\"snapshot\"}", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": "", + "title": "Running", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + }, + { + "op": "=", + "text": "No", + "value": "0" + }, + { + "op": "=", + "text": "Yes", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": false, + "colors": [ + "#d44a3a", + "rgba(237, 129, 40, 0.89)", + "#299c46" + ], + "datasource": "prometheus", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 2, + "w": 3, + "x": 9, + "y": 11 + }, + "id": 30, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_SnapshotCompleted{plugin=\"$connector_type\",name=\"$connector_name\",context=\"snapshot\"}", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": "", + "title": "Completed", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + }, + { + "op": "=", + "text": "No", + "value": "0" + }, + { + "op": "=", + "text": "Yes", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": false, + "colors": [ + "#d44a3a", + "rgba(237, 129, 40, 0.89)", + "#299c46" + ], + "datasource": "prometheus", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 2, + "w": 3, + "x": 12, + "y": 11 + }, + "id": 29, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_SnapshotAborted{plugin=\"$connector_type\",name=\"$connector_name\",context=\"snapshot\"}", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": "", + "title": "Aborted", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + }, + { + "op": "=", + "text": "No", + "value": "0" + }, + { + "op": "=", + "text": "Yes", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "prometheus", + "fill": 1, + "gridPos": { + "h": 9, + "w": 12, + "x": 0, + "y": 13 + }, + "id": 26, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_TotalNumberOfEventsSeen{plugin=\"$connector_type\",name=\"$connector_name\",context=\"snapshot\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Total events received", + "refId": "A" + }, + { + "expr": "debezium_metrics_NumberOfEventsSkipped{plugin=\"$connector_type\",name=\"$connector_name\",context=\"snapshot\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Events skipped", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Event count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "columns": [], + "datasource": "prometheus", + "fontSize": "100%", + "gridPos": { + "h": 9, + "w": 12, + "x": 12, + "y": 13 + }, + "id": 22, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "Time", + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "pattern": "Time", + "type": "date" + }, + { + "alias": "Table", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "table", + "thresholds": [], + "type": "string", + "unit": "short" + }, + { + "alias": "Rows", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 0, + "mappingType": 1, + "pattern": "Value", + "thresholds": [], + "type": "number", + "unit": "short" + }, + { + "alias": "", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "decimals": 2, + "pattern": "/.*/", + "thresholds": [], + "type": "hidden", + "unit": "short" + } + ], + "targets": [ + { + "expr": "debezium_metrics_RowsScanned{plugin=\"$connector_type\",name=\"$connector_name\"}", + "format": "table", + "hide": false, + "intervalFactor": 1, + "refId": "A" + } + ], + "title": "Rows scanned", + "transform": "table", + "type": "table" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 22 + }, + "id": 32, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "prometheus", + "fill": 1, + "gridPos": { + "h": 9, + "w": 12, + "x": 0, + "y": 3 + }, + "id": 34, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "kafka_connect_metrics_incoming_byte_rate", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Incoming({{client}})", + "refId": "A" + }, + { + "expr": "kafka_connect_metrics_outgoing_byte_rate", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Outgoing({{client}})", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Network traffic rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "Kafka Connect", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 23 + }, + "id": 20, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "prometheus", + "fill": 1, + "gridPos": { + "h": 9, + "w": 12, + "x": 0, + "y": 23 + }, + "id": 6, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "jvm_memory_bytes_used{area='heap'}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Used", + "refId": "A" + }, + { + "expr": "jvm_memory_bytes_max{area='heap'}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "Maximum", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Heap Memory", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "prometheus", + "fill": 1, + "gridPos": { + "h": 9, + "w": 12, + "x": 12, + "y": 23 + }, + "id": 24, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "jvm_threads_current", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Running", + "refId": "A" + }, + { + "expr": "jvm_threads_peak", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Maximum", + "refId": "B" + }, + { + "expr": "jvm_threads_started_total", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Started in total", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Threads", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "JVM", + "type": "row" + } + ], + "refresh": "5s", + "schemaVersion": 16, + "style": "dark", + "tags": [], + "templating": { + "list": [ + { + "allValue": null, + "current": { + "tags": [], + "text": "sql_server", + "value": "sql_server" + }, + "datasource": "prometheus", + "hide": 0, + "includeAll": false, + "label": "Connector Type", + "multi": false, + "name": "connector_type", + "options": [ + { + "selected": false, + "text": "mysql", + "value": "mysql" + }, + { + "selected": false, + "text": "postgres", + "value": "postgres" + }, + { + "selected": false, + "text": "mongodb", + "value": "mongodb" + }, + { + "selected": true, + "text": "sql_server", + "value": "sql_server" + }, + { + "selected": false, + "text": "oracle", + "value": "oracle" + }, + { + "selected": false, + "text": "db2", + "value": "db2" + } + ], + "query": "debezium_metrics_TotalNumberOfEventsSeen", + "refresh": 0, + "regex": "/.*plugin=\"([^\"]+)\".*/", + "skipUrlSync": false, + "sort": 0, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + }, + { + "allValue": null, + "current": { + "text": "server1", + "value": "server1" + }, + "datasource": "prometheus", + "hide": 0, + "includeAll": false, + "label": "Connector Name", + "multi": false, + "name": "connector_name", + "options": [ + { + "selected": true, + "text": "server1", + "value": "server1" + } + ], + "query": "debezium_metrics_TotalNumberOfEventsSeen{plugin=\"$connector_type\"}", + "refresh": 0, + "regex": "/.*[,\\(]name=\"([^\"]+)\".*/", + "skipUrlSync": false, + "sort": 0, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + } + ] + }, + "time": { + "from": "now-1h", + "to": "now" + }, + "timepicker": { + "refresh_intervals": [ + "5s", + "10s", + "30s", + "1m", + "5m", + "15m", + "30m", + "1h", + "2h", + "1d" + ], + "time_options": [ + "5m", + "15m", + "1h", + "6h", + "12h", + "24h", + "2d", + "7d", + "30d" + ] + }, + "timezone": "", + "title": "Debezium", + "uid": "7HrWpmEiz", + "version": 1 +} \ No newline at end of file diff --git a/deploy/config/grafana/dashboards/debezium/debezium-mysql-connector.json b/deploy/config/grafana/dashboards/debezium/debezium-mysql-connector.json new file mode 100644 index 000000000..6053012f9 --- /dev/null +++ b/deploy/config/grafana/dashboards/debezium/debezium-mysql-connector.json @@ -0,0 +1,3100 @@ +{ + "__inputs": [ + { + "name": "DS_PROMETHEUS", + "label": "Prometheus", + "description": "", + "type": "datasource", + "pluginId": "prometheus", + "pluginName": "Prometheus" + } + ], + "__requires": [ + { + "type": "panel", + "id": "bargauge", + "name": "Bar Gauge", + "version": "" + }, + { + "type": "panel", + "id": "gauge", + "name": "Gauge", + "version": "" + }, + { + "type": "grafana", + "id": "grafana", + "name": "Grafana", + "version": "6.5.2" + }, + { + "type": "panel", + "id": "graph", + "name": "Graph", + "version": "" + }, + { + "type": "datasource", + "id": "prometheus", + "name": "Prometheus", + "version": "1.0.0" + }, + { + "type": "panel", + "id": "singlestat", + "name": "Singlestat", + "version": "" + }, + { + "type": "panel", + "id": "text", + "name": "Text", + "version": "" + } + ], + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": "-- Grafana --", + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "type": "dashboard" + } + ] + }, + "description": "Monitoring Dashboard for Debezium MySQL connector", + "editable": true, + "gnetId": null, + "graphTooltip": 0, + "id": null, + "iteration": 1578408989490, + "links": [], + "panels": [ + { + "collapsed": false, + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 0 + }, + "id": 6, + "panels": [], + "title": "Binlog Metrics", + "type": "row" + }, + { + "content": "\n# Debezium MySQL Connector Metrics\n\nThe Debezium MySQL connector has three metric types in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect have.\n\n* snapshot metrics\n\n* binlog metrics\n\n* schema history metrics\n\nFor more detailed informarion about the metrics please visit the Debezium documentation page\n\n[Click here to visit](https://debezium.io/documentation/reference/1.0/assemblies/cdc-mysql-connector/as_deploy-the-mysql-connector.html#mysql-connector-binlog-metrics)\n\n\n\n", + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 1 + }, + "id": 76, + "mode": "markdown", + "options": {}, + "timeFrom": null, + "timeShift": null, + "title": "General Info", + "transparent": true, + "type": "text" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 12, + "y": 1 + }, + "id": 48, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_Connected{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Connected", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "NO", + "value": "0" + }, + { + "op": "=", + "text": "YES", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 16, + "y": 1 + }, + "id": 50, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_BinlogPosition{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Binlog Position", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#8F3BB8" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 20, + "y": 1 + }, + "id": 52, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_IsGtidModeEnabled{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "GTID Enabled", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "NO", + "value": "0" + }, + { + "op": "=", + "text": "YES", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 12, + "y": 5 + }, + "id": 68, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_NumberOfCommittedTransactions{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "NumberOf Committed Transactions", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 16, + "y": 5 + }, + "id": 70, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_NumberOfRolledBackTransactions{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "NumberOf RolledBack Transactions", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 20, + "y": 5 + }, + "id": 72, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_NumberOfNotWellFormedTransactions{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "NumberOf Not WellFormed Transactions", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 0, + "y": 9 + }, + "id": 60, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_TotalNumberOfEventsSeen{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Total NumberOf EventsSeen", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 4, + "w": 4, + "x": 4, + "y": 9 + }, + "id": 62, + "options": { + "fieldOptions": { + "calcs": [ + "mean" + ], + "defaults": { + "mappings": [], + "max": 1000, + "min": 0, + "thresholds": [ + { + "color": "rgba(89, 145, 255, 0.78)", + "value": null + }, + { + "color": "semi-dark-purple", + "value": 500 + }, + { + "color": "red", + "value": 800 + } + ] + }, + "override": {}, + "values": false + }, + "orientation": "auto", + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "6.5.2", + "targets": [ + { + "expr": "debezium_metrics_NumberOfSkippedEvents{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "NumberOf Skipped Events", + "type": "gauge" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 8, + "w": 4, + "x": 8, + "y": 9 + }, + "id": 64, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_NumberOfEventsFiltered{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "NumberOf Events Filtered", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "aliasColors": { + "debezium_metrics_SecondsSinceLastEvent{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-purple", + "debezium_metrics_SecondsSinceLastEvent{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "dark-purple", + "debezium_metrics_SecondsSinceLastEvent{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "dark-purple" + }, + "bars": true, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 9 + }, + "hiddenSeries": false, + "id": 54, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": false + }, + "lines": false, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pluginVersion": "6.5.2", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_SecondsSinceLastEvent{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Seconds Since LastEvent", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 0, + "y": 13 + }, + "id": 78, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_QueueTotalCapacity{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Queue Total Capacity", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 4, + "y": 13 + }, + "id": 80, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "pluginVersion": "6.5.2", + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_QueueRemainingCapacity{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Queue Remaining Capacity", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "aliasColors": { + "debezium_metrics_SecondsBehindMaster{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-blue", + "debezium_metrics_SecondsBehindMaster{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "semi-dark-purple", + "debezium_metrics_SecondsBehindMaster{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-blue" + }, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 17 + }, + "hiddenSeries": false, + "id": 56, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pluginVersion": "6.5.2", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_SecondsBehindMaster{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Seconds Behind Master", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": { + "debezium_metrics_MilliSecondsBehindSource{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-purple", + "debezium_metrics_MilliSecondsBehindSource{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "semi-dark-blue", + "debezium_metrics_MilliSecondsBehindSource{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "dark-purple" + }, + "bars": true, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 17 + }, + "hiddenSeries": false, + "id": 58, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": false, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_MilliSecondsBehindSource{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "MilliSeconds Behind Source", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": { + "debezium_metrics_NumberOfDisconnects{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-orange", + "debezium_metrics_NumberOfDisconnects{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "semi-dark-blue", + "debezium_metrics_NumberOfDisconnects{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "dark-purple" + }, + "bars": true, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 25 + }, + "hiddenSeries": false, + "id": 66, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_NumberOfDisconnects{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "NumberOf Disconnects", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": { + "debezium_metrics_NumberOfLargeTransactions{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-purple", + "debezium_metrics_NumberOfLargeTransactions{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "semi-dark-purple", + "debezium_metrics_NumberOfLargeTransactions{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "dark-blue" + }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 25 + }, + "hiddenSeries": false, + "id": 74, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_NumberOfLargeTransactions{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "NumberOf Large Transactions", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "collapsed": false, + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 33 + }, + "id": 2, + "panels": [], + "title": "Snapshot Metrics", + "type": "row" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorPrefix": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 0, + "y": 34 + }, + "id": 22, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_TotalTableCount{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Total Tablecount", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 4, + "w": 4, + "x": 4, + "y": 34 + }, + "id": 24, + "links": [], + "options": { + "fieldOptions": { + "calcs": [ + "mean" + ], + "defaults": { + "mappings": [ + { + "id": 0, + "op": "=", + "text": "N/A", + "type": 1, + "value": "null" + } + ], + "max": 600, + "min": 0, + "nullValueMode": "connected", + "thresholds": [ + { + "color": "semi-dark-blue", + "value": null + }, + { + "color": "semi-dark-purple", + "value": 200 + }, + { + "color": "red", + "value": 500 + } + ], + "unit": "none" + }, + "override": {}, + "values": false + }, + "orientation": "horizontal", + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "6.5.2", + "targets": [ + { + "expr": "debezium_metrics_RemainingTableCount{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Remaining Tales", + "type": "gauge" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "#3274D9", + "#299c46" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 8, + "y": 34 + }, + "id": 26, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_HoldingGlobalLock{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Holding GlobalLock", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "NO", + "value": "0" + }, + { + "op": "=", + "text": "YES", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 12, + "y": 34 + }, + "id": 28, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "pluginVersion": "6.5.2", + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_SnapshotRunning{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Running", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "NO", + "value": "0" + }, + { + "op": "=", + "text": "YES", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#299c46" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 16, + "y": 34 + }, + "id": 30, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_SnapshotAborted{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Aborted", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "NO", + "value": "0" + }, + { + "op": "=", + "text": "YES", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 20, + "y": 34 + }, + "id": 32, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_SnapshotCompleted{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Completed", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "NO", + "value": "0" + }, + { + "op": "=", + "text": "YES", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 4, + "w": 12, + "x": 0, + "y": 38 + }, + "id": 34, + "links": [], + "options": { + "displayMode": "lcd", + "fieldOptions": { + "calcs": [ + "mean" + ], + "defaults": { + "mappings": [], + "max": 18000, + "min": 0, + "thresholds": [ + { + "color": "semi-dark-blue", + "value": null + }, + { + "color": "semi-dark-purple", + "value": 9000 + }, + { + "color": "red", + "value": 15000 + } + ], + "unit": "s" + }, + "override": {}, + "values": false + }, + "orientation": "auto" + }, + "pluginVersion": "6.5.2", + "targets": [ + { + "expr": "debezium_metrics_SnapshotDurationInSeconds{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Duration", + "type": "bargauge" + }, + { + "aliasColors": { + "debezium_metrics_NumberOfEventsFiltered{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-blue", + "debezium_metrics_NumberOfEventsFiltered{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "dark-purple", + "debezium_metrics_NumberOfEventsFiltered{context=\"snapshot\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "dark-blue" + }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 38 + }, + "hiddenSeries": false, + "id": 38, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_NumberOfEventsFiltered{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "NumberOf Events Filtered", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "ms", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 0, + "y": 42 + }, + "id": 36, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_MilliSecondsSinceLastEvent{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "MilliSeconds Since LastEvent", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": false, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 4, + "y": 42 + }, + "id": 42, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_QueueTotalCapacity{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Queue Total Capacity", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 4, + "w": 4, + "x": 8, + "y": 42 + }, + "id": 44, + "options": { + "fieldOptions": { + "calcs": [ + "mean" + ], + "defaults": { + "mappings": [], + "max": 10000, + "min": 0, + "thresholds": [ + { + "color": "semi-dark-blue", + "value": null + }, + { + "color": "dark-purple", + "value": 7000 + }, + { + "color": "red", + "value": 8000 + } + ] + }, + "override": {}, + "values": false + }, + "orientation": "auto", + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "6.5.2", + "targets": [ + { + "expr": "debezium_metrics_QueueRemainingCapacity{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Queue Remaining Capacity", + "type": "gauge" + }, + { + "aliasColors": { + "debezium_metrics_NumberOfErroneousEvents{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-orange", + "debezium_metrics_NumberOfErroneousEvents{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-blue", + "debezium_metrics_NumberOfErroneousEvents{context=\"snapshot\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "semi-dark-purple" + }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 46 + }, + "hiddenSeries": false, + "id": 40, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_NumberOfErroneousEvents{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "NumberOf Erroneous Events", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": { + "debezium_metrics_TotalNumberOfEventsSeen{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-purple", + "debezium_metrics_TotalNumberOfEventsSeen{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-purple", + "debezium_metrics_TotalNumberOfEventsSeen{context=\"snapshot\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "semi-dark-blue" + }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 46 + }, + "hiddenSeries": false, + "id": 46, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_TotalNumberOfEventsSeen{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Total NumberOf Events Seen", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "collapsed": false, + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 54 + }, + "id": 8, + "panels": [], + "title": "Schema History Metrics", + "type": "row" + }, + { + "content": "\n# Schema Changes Metrics\n\nFor detailed explanation for all the schema changes metrics: [Visit Debezium documentation](https://debezium.io/documentation/reference/1.0/assemblies/cdc-mysql-connector/as_deploy-the-mysql-connector.html#mysql-connector-schema-history-metrics)\n\n\n\n\n", + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 5, + "w": 12, + "x": 0, + "y": 55 + }, + "id": 20, + "mode": "markdown", + "options": {}, + "timeFrom": null, + "timeShift": null, + "title": "Metric Title", + "transparent": true, + "type": "text" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": false, + "colors": [ + "#299c46", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "decimals": null, + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 5, + "w": 4, + "x": 12, + "y": 55 + }, + "id": 10, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_RecoveryStartTime{context=\"schema-history\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}", + "format": "time_series", + "instant": false, + "legendFormat": "", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Recovery StartTime", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "first" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": false, + "colors": [ + "#299c46", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "ms", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 5, + "w": 4, + "x": 16, + "y": 55 + }, + "id": 16, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_MilliSecondsSinceLastAppliedChange{context=\"schema-history\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Since Last AppliedChange", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": false, + "colors": [ + "#299c46", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "ms", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 5, + "w": 4, + "x": 20, + "y": 55 + }, + "id": 18, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_MilliSecondsSinceLastRecoveredChange{context=\"schema-history\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Since Last RecoveredChange", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "aliasColors": { + "debezium_metrics_ChangesRecovered{context=\"schema-history\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "dark-blue", + "debezium_metrics_ChangesRecovered{context=\"schema-history\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "dark-purple" + }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 60 + }, + "hiddenSeries": false, + "id": 12, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_ChangesRecovered{context=\"schema-history\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Changes Recovered", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": { + "debezium_metrics_ChangesApplied{context=\"schema-history\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "dark-purple", + "debezium_metrics_ChangesApplied{context=\"schema-history\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "dark-blue" + }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 60 + }, + "hiddenSeries": false, + "id": 14, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_ChangesApplied{context=\"schema-history\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Changes Applied", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "refresh": "5s", + "schemaVersion": 21, + "style": "dark", + "tags": [ + "mysql", + "debezium" + ], + "templating": { + "list": [ + { + "allValue": null, + "current": {}, + "datasource": "${DS_PROMETHEUS}", + "definition": "debezium_metrics_BinlogPosition", + "hide": 0, + "includeAll": true, + "label": "MySQL Node", + "multi": false, + "name": "name", + "options": [], + "query": "debezium_metrics_BinlogPosition", + "refresh": 1, + "regex": "/.*[,\\(]name=\"([^\"]+)\".*/", + "skipUrlSync": false, + "sort": 1, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + }, + { + "allValue": null, + "current": {}, + "datasource": "${DS_PROMETHEUS}", + "definition": "debezium_metrics_BinlogPosition", + "hide": 0, + "includeAll": false, + "label": "Connector Node", + "multi": false, + "name": "instance", + "options": [], + "query": "debezium_metrics_BinlogPosition", + "refresh": 1, + "regex": "/.*instance=\"([^\"]+)\".*/", + "skipUrlSync": false, + "sort": 0, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + }, + { + "allValue": null, + "current": {}, + "datasource": "${DS_PROMETHEUS}", + "definition": "debezium_metrics_MilliSecondsSinceLastEvent", + "hide": 0, + "includeAll": false, + "label": "context", + "multi": false, + "name": "context", + "options": [], + "query": "debezium_metrics_MilliSecondsSinceLastEvent", + "refresh": 1, + "regex": "/.*context=\"([^\"]+)\".*/", + "skipUrlSync": false, + "sort": 0, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + }, + { + "allValue": null, + "current": {}, + "datasource": "${DS_PROMETHEUS}", + "definition": "debezium_metrics_BinlogPosition", + "hide": 0, + "includeAll": false, + "label": "plugin", + "multi": false, + "name": "plugin", + "options": [], + "query": "debezium_metrics_BinlogPosition", + "refresh": 1, + "regex": "/.*plugin=\"([^\"]+)\".*/", + "skipUrlSync": false, + "sort": 1, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + } + ] + }, + "time": { + "from": "now-1h", + "to": "now" + }, + "timepicker": { + "refresh_intervals": [ + "5s", + "10s", + "30s", + "1m", + "5m", + "15m", + "30m", + "1h", + "2h", + "1d" + ] + }, + "timezone": "browser", + "title": "MySQL Connector", + "uid": "Ro1hBYYZz", + "version": 22 +} \ No newline at end of file diff --git a/deploy/config/prometheus.yml b/deploy/config/prometheus.yml index d6b7cc835..70435f26c 100644 --- a/deploy/config/prometheus.yml +++ b/deploy/config/prometheus.yml @@ -9,4 +9,12 @@ scrape_configs: - job_name: 'sink' static_configs: - targets: - - sink:8084 \ No newline at end of file + - sink:8084 + - job_name: redpanda + static_configs: + - targets: + - kafka:9644 + - job_name: debezium + static_configs: + - targets: + - debezium:8080 diff --git a/deploy/docker/docker-compose.yaml b/deploy/docker/docker-compose.yaml index e07c02710..6cb0ce750 100644 --- a/deploy/docker/docker-compose.yaml +++ b/deploy/docker/docker-compose.yaml @@ -97,10 +97,17 @@ services: debezium: container_name: debezium - image: debezium/connect:2.0 + #image: debezium/connect:2.0 + build: + context: ../../docker/debezium_jmx + args: + DEBEZIUM_VERSION: 2.0 + JMX_AGENT_VERSION: 0.15.0 restart: "no" ports: - "8083:8083" + - "1976:1976" + environment: # Where to find Kafka broker - BOOTSTRAP_SERVERS=kafka:9092 @@ -124,6 +131,9 @@ services: - CONNECT_VALUE_CONVERTER_APICURIO_REGISTRY_URL=http://schemaregistry:8080/apis/registry/v2 - CONNECT_VALUE_CONVERTER_APICURIO_REGISTRY_AUTO-REGISTER=true - CONNECT_VALUE_CONVERTER_APICURIO_REGISTRY_FIND-LATEST=true + - KAFKA_OPTS=-javaagent:/kafka/etc/jmx_prometheus_javaagent.jar=8080:/kafka/etc/config.yml + - JMXHOST=localhost + - JMXPORT=1976 #- LOG_LEVEL=DEBUG depends_on: - kafka @@ -236,14 +246,21 @@ services: - ../config/prometheus.yml:/opt/bitnami/prometheus/conf/prometheus.yml grafana: - container_name: grafana - image: grafana/grafana + build: + context: ../../docker/grafana + args: + GRAFANA_VERSION: 7.5.5 + #container_name: grafana + #image: grafana/grafana restart: "no" - volumes: - - ../config/grafana/dashboards:/etc/grafana/provisioning/dashboards + #volumes: + # - ../config/grafana/dashboards:/etc/grafana/provisioning/dashboards ports: - "3000:3000" - + links: + - prometheus + environment: + - DS_PROMETHEUS=prometheus #### END OF MONITORING ### bash: diff --git a/deploy/sink-connector-setup-sysbench.sh b/deploy/sink-connector-setup-sysbench.sh new file mode 100755 index 000000000..53b12216b --- /dev/null +++ b/deploy/sink-connector-setup-sysbench.sh @@ -0,0 +1,77 @@ +#!/bin/bash + +# Source configuration +CUR_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" >/dev/null 2>&1 && pwd)" +source "${CUR_DIR}/sink-connector-config.sh" + +# clickhouse-sink-connector params + +CLICKHOUSE_HOST="clickhouse" +CLICKHOUSE_PORT=8123 +CLICKHOUSE_USER="root" +CLICKHOUSE_PASSWORD="root" +CLICKHOUSE_TABLE="employees" +CLICKHOUSE_DATABASE="test" + +BUFFER_COUNT=10000 + +#SERVER5432.transaction +TOPICS="SERVER5432.sbtest.sbtest1" +TOPICS_TABLE_MAP="SERVER5432.test.employees_predated:employees, SERVER5432.test.products:products" +#TOPICS="SERVER5432" + +#"topics.regex": "SERVER5432.sbtest.(.*), SERVER5432.test.(.*)", + +#"topics": "${TOPICS}", + +cat <" >CH.tsv docker exec -it mysql-master mysql -uroot -proot -B -N -e "select * from sbtest.sbtest1 order by id" | grep -v "Using a password on the command line interface" >MySQL.tsv -diff MySQL.tsv CH.tsv \ No newline at end of file +diff --strip-trailing-cr MySQL.tsv CH.tsv \ No newline at end of file diff --git a/docker/debezium_jmx/Dockerfile b/docker/debezium_jmx/Dockerfile new file mode 100644 index 000000000..961e444a7 --- /dev/null +++ b/docker/debezium_jmx/Dockerfile @@ -0,0 +1,9 @@ +ARG DEBEZIUM_VERSION +FROM quay.io/debezium/connect:${DEBEZIUM_VERSION} + +ARG JMX_AGENT_VERSION +RUN mkdir /kafka/etc && cd /kafka/etc &&\ + curl -so jmx_prometheus_javaagent.jar \ + https://repo1.maven.org/maven2/io/prometheus/jmx/jmx_prometheus_javaagent/$JMX_AGENT_VERSION/jmx_prometheus_javaagent-$JMX_AGENT_VERSION.jar + +COPY config.yml /kafka/etc/config.yml \ No newline at end of file diff --git a/docker/debezium_jmx/config.yml b/docker/debezium_jmx/config.yml new file mode 100644 index 000000000..effdc8326 --- /dev/null +++ b/docker/debezium_jmx/config.yml @@ -0,0 +1,24 @@ +startDelaySeconds: 0 +ssl: false +lowercaseOutputName: false +lowercaseOutputLabelNames: false +rules: + - pattern : "kafka.connect([^:]+):" + name: "kafka_connect_worker_metrics_$1" + - pattern : "kafka.connect<>([^:]+)" + name: "kafka_connect_metrics_$2" + labels: + client: "$1" + - pattern: "debezium.([^:]+)]+)><>RowsScanned" + name: "debezium_metrics_RowsScanned" + labels: + plugin: "$1" + name: "$3" + context: "$2" + table: "$4" + - pattern: "debezium.([^:]+)]+)>([^:]+)" + name: "debezium_metrics_$4" + labels: + plugin: "$1" + name: "$3" + context: "$2" \ No newline at end of file diff --git a/docker/grafana/Dockerfile b/docker/grafana/Dockerfile new file mode 100644 index 000000000..1757d9020 --- /dev/null +++ b/docker/grafana/Dockerfile @@ -0,0 +1,6 @@ +ARG GRAFANA_VERSION +FROM grafana/grafana:${GRAFANA_VERSION} + +COPY ./config/dashboard.yml /etc/grafana/provisioning/dashboards +COPY ./config/datasource.yml /etc/grafana/provisioning/datasources +COPY ./config/debezium-dashboard.json /var/lib/grafana/dashboards/debezium-dashboard.json \ No newline at end of file diff --git a/docker/grafana/config/Datasource.yml b/docker/grafana/config/Datasource.yml new file mode 100644 index 000000000..beaa0b7e9 --- /dev/null +++ b/docker/grafana/config/Datasource.yml @@ -0,0 +1,10 @@ +datasources: + - access: 'proxy' # make grafana perform the requests + editable: true # whether it should be editable + is_default: true # whether this should be the default DS + name: 'Prometheus' # name of the datasource + org_id: 1 # id of the organization to tie this datasource to + type: 'prometheus' # type of the data source + url: 'http://prometheus:9090' # url of the prom instance + database: 'prometheus' + version: 1 # well, versioning \ No newline at end of file diff --git a/docker/grafana/config/dashboard.yml b/docker/grafana/config/dashboard.yml new file mode 100644 index 000000000..8673131f8 --- /dev/null +++ b/docker/grafana/config/dashboard.yml @@ -0,0 +1,10 @@ +apiVersion: 1 +providers: + - name: 'default' + orgId: 1 + folder: '' + type: file + disableDeletion: false + updateIntervalSeconds: 10 #how often Grafana will scan for changed dashboards + options: + path: /var/lib/grafana/dashboards \ No newline at end of file diff --git a/docker/grafana/config/datasource.yml b/docker/grafana/config/datasource.yml new file mode 100644 index 000000000..ba665915f --- /dev/null +++ b/docker/grafana/config/datasource.yml @@ -0,0 +1,8 @@ +apiVersion: 1 + +datasources: + - name: prometheus + type: prometheus + url: http://prometheus:9090 + access: proxy + version: 1 \ No newline at end of file diff --git a/docker/grafana/config/debezium-dashboard.json b/docker/grafana/config/debezium-dashboard.json new file mode 100644 index 000000000..67491a4a2 --- /dev/null +++ b/docker/grafana/config/debezium-dashboard.json @@ -0,0 +1,1364 @@ +{ + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": "-- Grafana --", + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "type": "dashboard" + } + ] + }, + "editable": true, + "gnetId": null, + "graphTooltip": 0, + "id": 1, + "iteration": 1544085491414, + "links": [], + "panels": [ + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 0 + }, + "id": 12, + "panels": [], + "title": "Streaming", + "type": "row" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorPostfix": false, + "colorPrefix": false, + "colorValue": true, + "colors": [ + "#299c46", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "prometheus", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": true, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 5, + "w": 3, + "x": 0, + "y": 1 + }, + "id": 14, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_MilliSecondsSinceLastEvent{plugin=\"$connector_type\",name=\"$connector_name\",context=~\"(binlog|streaming)\"}", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": "10000,60000", + "title": "Time since last event", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "current" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "prometheus", + "fill": 1, + "gridPos": { + "h": 9, + "w": 12, + "x": 3, + "y": 1 + }, + "id": 16, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_TotalNumberOfEventsSeen{plugin=\"$connector_type\",name=\"$connector_name\",context=~\"(binlog|streaming)\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Total events received", + "refId": "A" + }, + { + "expr": "debezium_metrics_NumberOfEventsSkipped{plugin=\"$connector_type\",name=\"$connector_name\",context=~\"(binlog|streaming)\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Events skipped", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Event count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#d44a3a", + "rgba(237, 129, 40, 0.89)", + "#299c46" + ], + "datasource": "prometheus", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 3, + "x": 0, + "y": 6 + }, + "id": 18, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_Connected{plugin=\"$connector_type\",name=\"$connector_name\",context=~\"(binlog|streaming)\"}", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": "0,1", + "title": "Connected", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + }, + { + "op": "=", + "text": "No", + "value": "0" + }, + { + "op": "=", + "text": "Yes", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 10 + }, + "id": 10, + "panels": [], + "title": "Snapshot", + "type": "row" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorPostfix": false, + "colorPrefix": false, + "colorValue": false, + "colors": [ + "#299c46", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "prometheus", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 2, + "w": 3, + "x": 0, + "y": 11 + }, + "id": 25, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_TotalTableCount{plugin=\"$connector_type\",name=\"$connector_name\",context=\"snapshot\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Total", + "refId": "A" + } + ], + "thresholds": "", + "title": "Table count", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorPostfix": false, + "colorPrefix": false, + "colorValue": false, + "colors": [ + "#299c46", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "prometheus", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 2, + "w": 3, + "x": 3, + "y": 11 + }, + "id": 28, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_RemainingTableCount{plugin=\"$connector_type\",name=\"$connector_name\",context=\"snapshot\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Remaining", + "refId": "B" + } + ], + "thresholds": "", + "title": "Remaining tables", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": false, + "colors": [ + "#d44a3a", + "rgba(237, 129, 40, 0.89)", + "#299c46" + ], + "datasource": "prometheus", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 2, + "w": 3, + "x": 6, + "y": 11 + }, + "id": 27, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_SnapshotRunning{plugin=\"$connector_type\",name=\"$connector_name\",context=\"snapshot\"}", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": "", + "title": "Running", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + }, + { + "op": "=", + "text": "No", + "value": "0" + }, + { + "op": "=", + "text": "Yes", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": false, + "colors": [ + "#d44a3a", + "rgba(237, 129, 40, 0.89)", + "#299c46" + ], + "datasource": "prometheus", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 2, + "w": 3, + "x": 9, + "y": 11 + }, + "id": 30, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_SnapshotCompleted{plugin=\"$connector_type\",name=\"$connector_name\",context=\"snapshot\"}", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": "", + "title": "Completed", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + }, + { + "op": "=", + "text": "No", + "value": "0" + }, + { + "op": "=", + "text": "Yes", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": false, + "colors": [ + "#d44a3a", + "rgba(237, 129, 40, 0.89)", + "#299c46" + ], + "datasource": "prometheus", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 2, + "w": 3, + "x": 12, + "y": 11 + }, + "id": 29, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_SnapshotAborted{plugin=\"$connector_type\",name=\"$connector_name\",context=\"snapshot\"}", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": "", + "title": "Aborted", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + }, + { + "op": "=", + "text": "No", + "value": "0" + }, + { + "op": "=", + "text": "Yes", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "prometheus", + "fill": 1, + "gridPos": { + "h": 9, + "w": 12, + "x": 0, + "y": 13 + }, + "id": 26, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_TotalNumberOfEventsSeen{plugin=\"$connector_type\",name=\"$connector_name\",context=\"snapshot\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Total events received", + "refId": "A" + }, + { + "expr": "debezium_metrics_NumberOfEventsSkipped{plugin=\"$connector_type\",name=\"$connector_name\",context=\"snapshot\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Events skipped", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Event count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "columns": [], + "datasource": "prometheus", + "fontSize": "100%", + "gridPos": { + "h": 9, + "w": 12, + "x": 12, + "y": 13 + }, + "id": 22, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "Time", + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "pattern": "Time", + "type": "date" + }, + { + "alias": "Table", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "table", + "thresholds": [], + "type": "string", + "unit": "short" + }, + { + "alias": "Rows", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 0, + "mappingType": 1, + "pattern": "Value", + "thresholds": [], + "type": "number", + "unit": "short" + }, + { + "alias": "", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "decimals": 2, + "pattern": "/.*/", + "thresholds": [], + "type": "hidden", + "unit": "short" + } + ], + "targets": [ + { + "expr": "debezium_metrics_RowsScanned{plugin=\"$connector_type\",name=\"$connector_name\"}", + "format": "table", + "hide": false, + "intervalFactor": 1, + "refId": "A" + } + ], + "title": "Rows scanned", + "transform": "table", + "type": "table" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 22 + }, + "id": 32, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "prometheus", + "fill": 1, + "gridPos": { + "h": 9, + "w": 12, + "x": 0, + "y": 3 + }, + "id": 34, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "kafka_connect_metrics_incoming_byte_rate", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Incoming({{client}})", + "refId": "A" + }, + { + "expr": "kafka_connect_metrics_outgoing_byte_rate", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Outgoing({{client}})", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Network traffic rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "Kafka Connect", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 23 + }, + "id": 20, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "prometheus", + "fill": 1, + "gridPos": { + "h": 9, + "w": 12, + "x": 0, + "y": 23 + }, + "id": 6, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "jvm_memory_bytes_used{area='heap'}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Used", + "refId": "A" + }, + { + "expr": "jvm_memory_bytes_max{area='heap'}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "Maximum", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Heap Memory", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "prometheus", + "fill": 1, + "gridPos": { + "h": 9, + "w": 12, + "x": 12, + "y": 23 + }, + "id": 24, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "jvm_threads_current", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Running", + "refId": "A" + }, + { + "expr": "jvm_threads_peak", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Maximum", + "refId": "B" + }, + { + "expr": "jvm_threads_started_total", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Started in total", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Threads", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "JVM", + "type": "row" + } + ], + "refresh": "5s", + "schemaVersion": 16, + "style": "dark", + "tags": [], + "templating": { + "list": [ + { + "allValue": null, + "current": { + "tags": [], + "text": "sql_server", + "value": "sql_server" + }, + "datasource": "prometheus", + "hide": 0, + "includeAll": false, + "label": "Connector Type", + "multi": false, + "name": "connector_type", + "options": [ + { + "selected": false, + "text": "mysql", + "value": "mysql" + }, + { + "selected": false, + "text": "postgres", + "value": "postgres" + }, + { + "selected": false, + "text": "mongodb", + "value": "mongodb" + }, + { + "selected": true, + "text": "sql_server", + "value": "sql_server" + }, + { + "selected": false, + "text": "oracle", + "value": "oracle" + }, + { + "selected": false, + "text": "db2", + "value": "db2" + } + ], + "query": "debezium_metrics_TotalNumberOfEventsSeen", + "refresh": 0, + "regex": "/.*plugin=\"([^\"]+)\".*/", + "skipUrlSync": false, + "sort": 0, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + }, + { + "allValue": null, + "current": { + "text": "server1", + "value": "server1" + }, + "datasource": "prometheus", + "hide": 0, + "includeAll": false, + "label": "Connector Name", + "multi": false, + "name": "connector_name", + "options": [ + { + "selected": true, + "text": "server1", + "value": "server1" + } + ], + "query": "debezium_metrics_TotalNumberOfEventsSeen{plugin=\"$connector_type\"}", + "refresh": 0, + "regex": "/.*[,\\(]name=\"([^\"]+)\".*/", + "skipUrlSync": false, + "sort": 0, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + } + ] + }, + "time": { + "from": "now-1h", + "to": "now" + }, + "timepicker": { + "refresh_intervals": [ + "5s", + "10s", + "30s", + "1m", + "5m", + "15m", + "30m", + "1h", + "2h", + "1d" + ], + "time_options": [ + "5m", + "15m", + "1h", + "6h", + "12h", + "24h", + "2d", + "7d", + "30d" + ] + }, + "timezone": "", + "title": "Debezium", + "uid": "7HrWpmEiz", + "version": 1 +} \ No newline at end of file diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnable.java b/src/main/java/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnable.java index 050a1fe07..ed608378a 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnable.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnable.java @@ -129,15 +129,15 @@ public DbWriter getDbWriterForTable(String topicName, String tableName, ClickHou // Check if DB instance exists for the current topic // or else create a new one. - if (this.topicToDbWriterMap.containsKey(topicName)) { - writer = this.topicToDbWriterMap.get(topicName); - } else { +// if (this.topicToDbWriterMap.containsKey(topicName)) { +// writer = this.topicToDbWriterMap.get(topicName); +// } else { writer = new DbWriter(this.dbCredentials.getHostName(), this.dbCredentials.getPort(), this.dbCredentials.getDatabase(), tableName, this.dbCredentials.getUserName(), this.dbCredentials.getPassword(), this.config, record); this.topicToDbWriterMap.put(topicName, writer); - } - +// } +// return writer; } // diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/model/ClickHouseStruct.java b/src/main/java/com/altinity/clickhouse/sink/connector/model/ClickHouseStruct.java index 0fb674e19..5f77a7abe 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/model/ClickHouseStruct.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/model/ClickHouseStruct.java @@ -209,16 +209,6 @@ public void setAdditionalMetaData(Map convertedValue) { } } - public int getTransactionId(String gtiId) { - int result = -1; - - String[] gtidArray = (gtiId).split(":"); - if(gtidArray.length == 2) { - result = Integer.parseInt(gtidArray[1]); - } - - return result; - } @Override public String toString() { return new StringBuffer() From 7572f29f9e383fc56424a72f30296c26dc9e0f7c Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Tue, 26 Jul 2022 17:06:05 -0400 Subject: [PATCH 05/13] Added logic to persist metrics and sink connector grafana dashboard. --- ...ebezium-connector-setup-schema-registry.sh | 4 +- deploy/debezium-connector-setup-sysbench.sh | 4 +- deploy/docker/docker-compose.yaml | 29 +- .../sink-connector-setup-schema-registry.sh | 2 +- docker/grafana/Dockerfile | 4 +- .../config/altinity_sink_connector.json | 274 ++ .../config/debezium-mysql-connector.json | 3100 +++++++++++++++++ .../sink/connector/common/Metrics.java | 42 +- .../sink/connector/db/DbWriter.java | 10 +- .../executor/ClickHouseBatchRunnable.java | 16 +- .../sink/connector/model/BlockMetaData.java | 35 + 11 files changed, 3499 insertions(+), 21 deletions(-) create mode 100644 docker/grafana/config/altinity_sink_connector.json create mode 100644 docker/grafana/config/debezium-mysql-connector.json diff --git a/deploy/debezium-connector-setup-schema-registry.sh b/deploy/debezium-connector-setup-schema-registry.sh index daa427b44..18b9f79dc 100755 --- a/deploy/debezium-connector-setup-schema-registry.sh +++ b/deploy/debezium-connector-setup-schema-registry.sh @@ -97,9 +97,9 @@ cat <\n \n

Altinity Sink Connector for ClickHouse

\n

For more information, visit ", + "mode": "html" + }, + "pluginVersion": "8.2.6", + "title": "Altinity Sink Connector for ClickHouse", + "type": "text" + }, + { + "datasource": "prometheus", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 5, + "x": 12, + "y": 0 + }, + "id": 4, + "options": { + "colorMode": "value", + "graphMode": "none", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "text": {}, + "textMode": "auto" + }, + "pluginVersion": "8.2.6", + "targets": [ + { + "exemplar": true, + "expr": "clickhouse_sink_binlog_pos{}", + "interval": "", + "legendFormat": "", + "refId": "A" + } + ], + "title": "Bin Log position", + "type": "stat" + }, + { + "datasource": "prometheus", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 5, + "x": 17, + "y": 0 + }, + "id": 6, + "options": { + "colorMode": "value", + "graphMode": "none", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "text": {}, + "textMode": "auto" + }, + "pluginVersion": "8.2.6", + "targets": [ + { + "exemplar": true, + "expr": "clickhouse_sink_gtid{}", + "interval": "", + "legendFormat": "", + "refId": "A" + } + ], + "title": "GTID - Transaction Id", + "type": "stat" + }, + { + "datasource": "prometheus", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "displayMode": "auto", + "filterable": false + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "topic" + }, + "properties": [ + { + "id": "custom.width", + "value": 324 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "job" + }, + "properties": [ + { + "id": "custom.width", + "value": 72 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "instance" + }, + "properties": [ + { + "id": "custom.width", + "value": 109 + } + ] + } + ] + }, + "gridPos": { + "h": 5, + "w": 22, + "x": 0, + "y": 5 + }, + "id": 8, + "options": { + "frameIndex": 0, + "showHeader": true, + "sortBy": [] + }, + "pluginVersion": "8.2.6", + "targets": [ + { + "exemplar": true, + "expr": "clickhouse_topics_num_records_total{}", + "format": "table", + "instant": true, + "interval": "", + "legendFormat": "", + "refId": "A" + } + ], + "title": "Records (Topic)", + "type": "table" + } + ], + "refresh": false, + "schemaVersion": 32, + "style": "dark", + "tags": [], + "templating": { + "list": [] + }, + "time": { + "from": "now-5m", + "to": "now" + }, + "timepicker": {}, + "timezone": "", + "title": "altinity_sink_connector_clickhouse", + "uid": "c4cKtgk4k", + "version": 1 +} \ No newline at end of file diff --git a/docker/grafana/config/debezium-mysql-connector.json b/docker/grafana/config/debezium-mysql-connector.json new file mode 100644 index 000000000..6053012f9 --- /dev/null +++ b/docker/grafana/config/debezium-mysql-connector.json @@ -0,0 +1,3100 @@ +{ + "__inputs": [ + { + "name": "DS_PROMETHEUS", + "label": "Prometheus", + "description": "", + "type": "datasource", + "pluginId": "prometheus", + "pluginName": "Prometheus" + } + ], + "__requires": [ + { + "type": "panel", + "id": "bargauge", + "name": "Bar Gauge", + "version": "" + }, + { + "type": "panel", + "id": "gauge", + "name": "Gauge", + "version": "" + }, + { + "type": "grafana", + "id": "grafana", + "name": "Grafana", + "version": "6.5.2" + }, + { + "type": "panel", + "id": "graph", + "name": "Graph", + "version": "" + }, + { + "type": "datasource", + "id": "prometheus", + "name": "Prometheus", + "version": "1.0.0" + }, + { + "type": "panel", + "id": "singlestat", + "name": "Singlestat", + "version": "" + }, + { + "type": "panel", + "id": "text", + "name": "Text", + "version": "" + } + ], + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": "-- Grafana --", + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "type": "dashboard" + } + ] + }, + "description": "Monitoring Dashboard for Debezium MySQL connector", + "editable": true, + "gnetId": null, + "graphTooltip": 0, + "id": null, + "iteration": 1578408989490, + "links": [], + "panels": [ + { + "collapsed": false, + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 0 + }, + "id": 6, + "panels": [], + "title": "Binlog Metrics", + "type": "row" + }, + { + "content": "\n# Debezium MySQL Connector Metrics\n\nThe Debezium MySQL connector has three metric types in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect have.\n\n* snapshot metrics\n\n* binlog metrics\n\n* schema history metrics\n\nFor more detailed informarion about the metrics please visit the Debezium documentation page\n\n[Click here to visit](https://debezium.io/documentation/reference/1.0/assemblies/cdc-mysql-connector/as_deploy-the-mysql-connector.html#mysql-connector-binlog-metrics)\n\n\n\n", + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 1 + }, + "id": 76, + "mode": "markdown", + "options": {}, + "timeFrom": null, + "timeShift": null, + "title": "General Info", + "transparent": true, + "type": "text" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 12, + "y": 1 + }, + "id": 48, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_Connected{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Connected", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "NO", + "value": "0" + }, + { + "op": "=", + "text": "YES", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 16, + "y": 1 + }, + "id": 50, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_BinlogPosition{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Binlog Position", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#8F3BB8" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 20, + "y": 1 + }, + "id": 52, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_IsGtidModeEnabled{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "GTID Enabled", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "NO", + "value": "0" + }, + { + "op": "=", + "text": "YES", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 12, + "y": 5 + }, + "id": 68, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_NumberOfCommittedTransactions{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "NumberOf Committed Transactions", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 16, + "y": 5 + }, + "id": 70, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_NumberOfRolledBackTransactions{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "NumberOf RolledBack Transactions", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 20, + "y": 5 + }, + "id": 72, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_NumberOfNotWellFormedTransactions{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "NumberOf Not WellFormed Transactions", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 0, + "y": 9 + }, + "id": 60, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_TotalNumberOfEventsSeen{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Total NumberOf EventsSeen", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 4, + "w": 4, + "x": 4, + "y": 9 + }, + "id": 62, + "options": { + "fieldOptions": { + "calcs": [ + "mean" + ], + "defaults": { + "mappings": [], + "max": 1000, + "min": 0, + "thresholds": [ + { + "color": "rgba(89, 145, 255, 0.78)", + "value": null + }, + { + "color": "semi-dark-purple", + "value": 500 + }, + { + "color": "red", + "value": 800 + } + ] + }, + "override": {}, + "values": false + }, + "orientation": "auto", + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "6.5.2", + "targets": [ + { + "expr": "debezium_metrics_NumberOfSkippedEvents{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "NumberOf Skipped Events", + "type": "gauge" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 8, + "w": 4, + "x": 8, + "y": 9 + }, + "id": 64, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_NumberOfEventsFiltered{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "NumberOf Events Filtered", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "aliasColors": { + "debezium_metrics_SecondsSinceLastEvent{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-purple", + "debezium_metrics_SecondsSinceLastEvent{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "dark-purple", + "debezium_metrics_SecondsSinceLastEvent{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "dark-purple" + }, + "bars": true, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 9 + }, + "hiddenSeries": false, + "id": 54, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": false + }, + "lines": false, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pluginVersion": "6.5.2", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_SecondsSinceLastEvent{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Seconds Since LastEvent", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 0, + "y": 13 + }, + "id": 78, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_QueueTotalCapacity{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Queue Total Capacity", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 4, + "y": 13 + }, + "id": 80, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "pluginVersion": "6.5.2", + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_QueueRemainingCapacity{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Queue Remaining Capacity", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "aliasColors": { + "debezium_metrics_SecondsBehindMaster{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-blue", + "debezium_metrics_SecondsBehindMaster{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "semi-dark-purple", + "debezium_metrics_SecondsBehindMaster{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-blue" + }, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 17 + }, + "hiddenSeries": false, + "id": 56, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pluginVersion": "6.5.2", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_SecondsBehindMaster{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Seconds Behind Master", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": { + "debezium_metrics_MilliSecondsBehindSource{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-purple", + "debezium_metrics_MilliSecondsBehindSource{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "semi-dark-blue", + "debezium_metrics_MilliSecondsBehindSource{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "dark-purple" + }, + "bars": true, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 17 + }, + "hiddenSeries": false, + "id": 58, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": false, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_MilliSecondsBehindSource{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "MilliSeconds Behind Source", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": { + "debezium_metrics_NumberOfDisconnects{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-orange", + "debezium_metrics_NumberOfDisconnects{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "semi-dark-blue", + "debezium_metrics_NumberOfDisconnects{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "dark-purple" + }, + "bars": true, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 25 + }, + "hiddenSeries": false, + "id": 66, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_NumberOfDisconnects{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "NumberOf Disconnects", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": { + "debezium_metrics_NumberOfLargeTransactions{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-purple", + "debezium_metrics_NumberOfLargeTransactions{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "semi-dark-purple", + "debezium_metrics_NumberOfLargeTransactions{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "dark-blue" + }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 25 + }, + "hiddenSeries": false, + "id": 74, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_NumberOfLargeTransactions{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "NumberOf Large Transactions", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "collapsed": false, + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 33 + }, + "id": 2, + "panels": [], + "title": "Snapshot Metrics", + "type": "row" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorPrefix": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 0, + "y": 34 + }, + "id": 22, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_TotalTableCount{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Total Tablecount", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 4, + "w": 4, + "x": 4, + "y": 34 + }, + "id": 24, + "links": [], + "options": { + "fieldOptions": { + "calcs": [ + "mean" + ], + "defaults": { + "mappings": [ + { + "id": 0, + "op": "=", + "text": "N/A", + "type": 1, + "value": "null" + } + ], + "max": 600, + "min": 0, + "nullValueMode": "connected", + "thresholds": [ + { + "color": "semi-dark-blue", + "value": null + }, + { + "color": "semi-dark-purple", + "value": 200 + }, + { + "color": "red", + "value": 500 + } + ], + "unit": "none" + }, + "override": {}, + "values": false + }, + "orientation": "horizontal", + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "6.5.2", + "targets": [ + { + "expr": "debezium_metrics_RemainingTableCount{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Remaining Tales", + "type": "gauge" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "#3274D9", + "#299c46" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 8, + "y": 34 + }, + "id": 26, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_HoldingGlobalLock{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Holding GlobalLock", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "NO", + "value": "0" + }, + { + "op": "=", + "text": "YES", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 12, + "y": 34 + }, + "id": 28, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "pluginVersion": "6.5.2", + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_SnapshotRunning{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Running", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "NO", + "value": "0" + }, + { + "op": "=", + "text": "YES", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#299c46" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 16, + "y": 34 + }, + "id": 30, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_SnapshotAborted{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Aborted", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "NO", + "value": "0" + }, + { + "op": "=", + "text": "YES", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 20, + "y": 34 + }, + "id": 32, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_SnapshotCompleted{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Completed", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "NO", + "value": "0" + }, + { + "op": "=", + "text": "YES", + "value": "1" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 4, + "w": 12, + "x": 0, + "y": 38 + }, + "id": 34, + "links": [], + "options": { + "displayMode": "lcd", + "fieldOptions": { + "calcs": [ + "mean" + ], + "defaults": { + "mappings": [], + "max": 18000, + "min": 0, + "thresholds": [ + { + "color": "semi-dark-blue", + "value": null + }, + { + "color": "semi-dark-purple", + "value": 9000 + }, + { + "color": "red", + "value": 15000 + } + ], + "unit": "s" + }, + "override": {}, + "values": false + }, + "orientation": "auto" + }, + "pluginVersion": "6.5.2", + "targets": [ + { + "expr": "debezium_metrics_SnapshotDurationInSeconds{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Duration", + "type": "bargauge" + }, + { + "aliasColors": { + "debezium_metrics_NumberOfEventsFiltered{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-blue", + "debezium_metrics_NumberOfEventsFiltered{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "dark-purple", + "debezium_metrics_NumberOfEventsFiltered{context=\"snapshot\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "dark-blue" + }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 38 + }, + "hiddenSeries": false, + "id": 38, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_NumberOfEventsFiltered{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "NumberOf Events Filtered", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": true, + "colors": [ + "#A352CC", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "ms", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 0, + "y": 42 + }, + "id": 36, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_MilliSecondsSinceLastEvent{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "MilliSeconds Since LastEvent", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": false, + "colors": [ + "#3274D9", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 4, + "y": 42 + }, + "id": 42, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_QueueTotalCapacity{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Queue Total Capacity", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 4, + "w": 4, + "x": 8, + "y": 42 + }, + "id": 44, + "options": { + "fieldOptions": { + "calcs": [ + "mean" + ], + "defaults": { + "mappings": [], + "max": 10000, + "min": 0, + "thresholds": [ + { + "color": "semi-dark-blue", + "value": null + }, + { + "color": "dark-purple", + "value": 7000 + }, + { + "color": "red", + "value": 8000 + } + ] + }, + "override": {}, + "values": false + }, + "orientation": "auto", + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "6.5.2", + "targets": [ + { + "expr": "debezium_metrics_QueueRemainingCapacity{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Queue Remaining Capacity", + "type": "gauge" + }, + { + "aliasColors": { + "debezium_metrics_NumberOfErroneousEvents{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-orange", + "debezium_metrics_NumberOfErroneousEvents{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-blue", + "debezium_metrics_NumberOfErroneousEvents{context=\"snapshot\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "semi-dark-purple" + }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 46 + }, + "hiddenSeries": false, + "id": 40, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_NumberOfErroneousEvents{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "NumberOf Erroneous Events", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": { + "debezium_metrics_TotalNumberOfEventsSeen{context=\"binlog\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-purple", + "debezium_metrics_TotalNumberOfEventsSeen{context=\"binlog\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "semi-dark-purple", + "debezium_metrics_TotalNumberOfEventsSeen{context=\"snapshot\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "semi-dark-blue" + }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 46 + }, + "hiddenSeries": false, + "id": 46, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_TotalNumberOfEventsSeen{context=\"$context\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Total NumberOf Events Seen", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "collapsed": false, + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 54 + }, + "id": 8, + "panels": [], + "title": "Schema History Metrics", + "type": "row" + }, + { + "content": "\n# Schema Changes Metrics\n\nFor detailed explanation for all the schema changes metrics: [Visit Debezium documentation](https://debezium.io/documentation/reference/1.0/assemblies/cdc-mysql-connector/as_deploy-the-mysql-connector.html#mysql-connector-schema-history-metrics)\n\n\n\n\n", + "datasource": "${DS_PROMETHEUS}", + "gridPos": { + "h": 5, + "w": 12, + "x": 0, + "y": 55 + }, + "id": 20, + "mode": "markdown", + "options": {}, + "timeFrom": null, + "timeShift": null, + "title": "Metric Title", + "transparent": true, + "type": "text" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": false, + "colors": [ + "#299c46", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "decimals": null, + "format": "none", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 5, + "w": 4, + "x": 12, + "y": 55 + }, + "id": 10, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_RecoveryStartTime{context=\"schema-history\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}", + "format": "time_series", + "instant": false, + "legendFormat": "", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Recovery StartTime", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "first" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": false, + "colors": [ + "#299c46", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "ms", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 5, + "w": 4, + "x": 16, + "y": 55 + }, + "id": 16, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_MilliSecondsSinceLastAppliedChange{context=\"schema-history\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Since Last AppliedChange", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "cacheTimeout": null, + "colorBackground": false, + "colorValue": false, + "colors": [ + "#299c46", + "rgba(237, 129, 40, 0.89)", + "#d44a3a" + ], + "datasource": "${DS_PROMETHEUS}", + "format": "ms", + "gauge": { + "maxValue": 100, + "minValue": 0, + "show": false, + "thresholdLabels": false, + "thresholdMarkers": true + }, + "gridPos": { + "h": 5, + "w": 4, + "x": 20, + "y": 55 + }, + "id": 18, + "interval": null, + "links": [], + "mappingType": 1, + "mappingTypes": [ + { + "name": "value to text", + "value": 1 + }, + { + "name": "range to text", + "value": 2 + } + ], + "maxDataPoints": 100, + "nullPointMode": "connected", + "nullText": null, + "options": {}, + "postfix": "", + "postfixFontSize": "50%", + "prefix": "", + "prefixFontSize": "50%", + "rangeMaps": [ + { + "from": "null", + "text": "N/A", + "to": "null" + } + ], + "sparkline": { + "fillColor": "rgba(31, 118, 189, 0.18)", + "full": false, + "lineColor": "rgb(31, 120, 193)", + "show": false, + "ymax": null, + "ymin": null + }, + "tableColumn": "", + "targets": [ + { + "expr": "debezium_metrics_MilliSecondsSinceLastRecoveredChange{context=\"schema-history\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": "", + "timeFrom": null, + "timeShift": null, + "title": "Since Last RecoveredChange", + "type": "singlestat", + "valueFontSize": "80%", + "valueMaps": [ + { + "op": "=", + "text": "N/A", + "value": "null" + } + ], + "valueName": "avg" + }, + { + "aliasColors": { + "debezium_metrics_ChangesRecovered{context=\"schema-history\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "dark-blue", + "debezium_metrics_ChangesRecovered{context=\"schema-history\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "dark-purple" + }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 60 + }, + "hiddenSeries": false, + "id": 12, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_ChangesRecovered{context=\"schema-history\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Changes Recovered", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": { + "debezium_metrics_ChangesApplied{context=\"schema-history\",instance=\"192.168.11.182:7071\",job=\"debezium\",name=\"snapshot-prod-aurora-cluster-cluster\",plugin=\"mysql\"}": "dark-purple", + "debezium_metrics_ChangesApplied{context=\"schema-history\",instance=\"192.168.13.124:7071\",job=\"debezium\",name=\"prod-aurora-cluster\",plugin=\"mysql\"}": "dark-blue" + }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_PROMETHEUS}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 60 + }, + "hiddenSeries": false, + "id": 14, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "debezium_metrics_ChangesApplied{context=\"schema-history\",instance=\"$instance\",job=\"debezium\",name=\"$name\",plugin=\"$plugin\"}\t\r", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Changes Applied", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "refresh": "5s", + "schemaVersion": 21, + "style": "dark", + "tags": [ + "mysql", + "debezium" + ], + "templating": { + "list": [ + { + "allValue": null, + "current": {}, + "datasource": "${DS_PROMETHEUS}", + "definition": "debezium_metrics_BinlogPosition", + "hide": 0, + "includeAll": true, + "label": "MySQL Node", + "multi": false, + "name": "name", + "options": [], + "query": "debezium_metrics_BinlogPosition", + "refresh": 1, + "regex": "/.*[,\\(]name=\"([^\"]+)\".*/", + "skipUrlSync": false, + "sort": 1, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + }, + { + "allValue": null, + "current": {}, + "datasource": "${DS_PROMETHEUS}", + "definition": "debezium_metrics_BinlogPosition", + "hide": 0, + "includeAll": false, + "label": "Connector Node", + "multi": false, + "name": "instance", + "options": [], + "query": "debezium_metrics_BinlogPosition", + "refresh": 1, + "regex": "/.*instance=\"([^\"]+)\".*/", + "skipUrlSync": false, + "sort": 0, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + }, + { + "allValue": null, + "current": {}, + "datasource": "${DS_PROMETHEUS}", + "definition": "debezium_metrics_MilliSecondsSinceLastEvent", + "hide": 0, + "includeAll": false, + "label": "context", + "multi": false, + "name": "context", + "options": [], + "query": "debezium_metrics_MilliSecondsSinceLastEvent", + "refresh": 1, + "regex": "/.*context=\"([^\"]+)\".*/", + "skipUrlSync": false, + "sort": 0, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + }, + { + "allValue": null, + "current": {}, + "datasource": "${DS_PROMETHEUS}", + "definition": "debezium_metrics_BinlogPosition", + "hide": 0, + "includeAll": false, + "label": "plugin", + "multi": false, + "name": "plugin", + "options": [], + "query": "debezium_metrics_BinlogPosition", + "refresh": 1, + "regex": "/.*plugin=\"([^\"]+)\".*/", + "skipUrlSync": false, + "sort": 1, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + } + ] + }, + "time": { + "from": "now-1h", + "to": "now" + }, + "timepicker": { + "refresh_intervals": [ + "5s", + "10s", + "30s", + "1m", + "5m", + "15m", + "30m", + "1h", + "2h", + "1d" + ] + }, + "timezone": "browser", + "title": "MySQL Connector", + "uid": "Ro1hBYYZz", + "version": 22 +} \ No newline at end of file diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/common/Metrics.java b/src/main/java/com/altinity/clickhouse/sink/connector/common/Metrics.java index 94a7f0f1d..4b535a893 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/common/Metrics.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/common/Metrics.java @@ -1,13 +1,16 @@ package com.altinity.clickhouse.sink.connector.common; +import com.altinity.clickhouse.sink.connector.model.BlockMetaData; import com.codahale.metrics.ConsoleReporter; import com.codahale.metrics.MetricRegistry; import com.codahale.metrics.Timer; import com.sun.net.httpserver.HttpServer; +import io.micrometer.core.instrument.Clock; import io.micrometer.core.instrument.Counter; -import io.micrometer.core.instrument.Gauge; import io.micrometer.prometheus.PrometheusConfig; import io.micrometer.prometheus.PrometheusMeterRegistry; +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.Gauge; import org.apache.commons.lang3.tuple.MutablePair; import org.slf4j.Logger; @@ -28,6 +31,8 @@ public class Metrics { private static ConsoleReporter reporter = null; + private static CollectorRegistry collectorRegistry; + private static PrometheusMeterRegistry meterRegistry; private static Gauge clickHouseSinkRecordsGauge; @@ -42,6 +47,13 @@ public class Metrics { private static Counter.Builder maxConsumerLagCounter; + + private static Counter.Builder topicsNumRecordsCounter; + + private static Gauge maxBinLogPositionCounter; + + private static Gauge gtidCounter; + private static HttpServer server; private static boolean enableMetrics = false; @@ -62,12 +74,13 @@ public static void initialize(String enableFlag, String metricsPort) { parseConfiguration(enableFlag, metricsPort); if(enableMetrics) { + collectorRegistry = new CollectorRegistry(); meterRegistry = - new PrometheusMeterRegistry(PrometheusConfig.DEFAULT); + new PrometheusMeterRegistry(PrometheusConfig.DEFAULT, collectorRegistry, Clock.SYSTEM); exposePrometheusPort(meterRegistry); - registerMetrics(); + registerMetrics(collectorRegistry); } } @@ -90,7 +103,7 @@ private static void parseConfiguration(String enableFlag, String metricsPort) { } } - private static void registerMetrics() { + private static void registerMetrics(CollectorRegistry collectorRegistry) { clickHouseSinkRecordsCounter = Counter.builder("clickhouse.sink.records"); @@ -100,6 +113,11 @@ private static void registerMetrics() { minConsumerLagCounter = Counter.builder("clickhouse.consumer.lag.min"); maxConsumerLagCounter = Counter.builder("clickhouse.consumer.lag.max"); + maxBinLogPositionCounter = Gauge.build().name("clickhouse_sink_binlog_pos").help("Bin Log Position").register(collectorRegistry); + gtidCounter = Gauge.build().name("clickhouse_sink_gtid").help("GTID Transaction Id").register(collectorRegistry); + + topicsNumRecordsCounter = Counter.builder("clickhouse.topics.num.records"); + } private static void exposePrometheusPort(PrometheusMeterRegistry prometheusMeterRegistry) { @@ -112,6 +130,7 @@ private static void exposePrometheusPort(PrometheusMeterRegistry prometheusMeter try (OutputStream os = httpExchange.getResponseBody()) { os.write(response.getBytes()); } + }); new Thread(server::start).start(); @@ -131,6 +150,16 @@ public static MetricRegistry registry() { public static Counter.Builder getClickHouseSinkRecordsCounter() { return clickHouseSinkRecordsCounter;} + + public static void updateMetrics(BlockMetaData bmd) { + maxBinLogPositionCounter.set(bmd.getBinLogPosition()); + //tag("partition", Integer.toString(bmd.getPartition())). + + gtidCounter.set(bmd.getTransactionId()); + //tag("partition", Integer.toString(bmd.getPartition())). + //register(Metrics.meterRegistry()).increment(bmd.getTransactionId()); + } + public static void updateSinkRecordsCounter(String blockUUid, Long taskId, String topicName, String tableName, HashMap> partitionToOffsetMap, int numRecords, long minSourceLag, long maxSourceLag, @@ -175,4 +204,9 @@ public static Timer timer(String first, String... keys) { return registry.timer(MetricRegistry.name(first, keys)); } + public static void updateCounters(String topicName, int numRecords) { + topicsNumRecordsCounter + .tag("topic", topicName).register(Metrics.meterRegistry()).increment(numRecords); + + } } diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java b/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java index d25e86538..d209cd0c2 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java @@ -2,11 +2,13 @@ import com.altinity.clickhouse.sink.connector.ClickHouseSinkConnectorConfig; import com.altinity.clickhouse.sink.connector.ClickHouseSinkConnectorConfigVariables; +import com.altinity.clickhouse.sink.connector.common.Metrics; import com.altinity.clickhouse.sink.connector.converters.ClickHouseConverter; import com.altinity.clickhouse.sink.connector.converters.DebeziumConverter; import com.altinity.clickhouse.sink.connector.db.operations.ClickHouseAlterTable; import com.altinity.clickhouse.sink.connector.db.operations.ClickHouseAutoCreateTable; import com.altinity.clickhouse.sink.connector.metadata.TableMetaDataWriter; +import com.altinity.clickhouse.sink.connector.model.BlockMetaData; import com.altinity.clickhouse.sink.connector.model.CdcRecordState; import com.altinity.clickhouse.sink.connector.model.ClickHouseStruct; import com.altinity.clickhouse.sink.connector.model.KafkaMetaData; @@ -363,7 +365,8 @@ public Map insert(ConcurrentLinkedQueue * * @param queryToRecordsMap */ - public void addToPreparedStatementBatch(Map>, List> queryToRecordsMap) { + public BlockMetaData addToPreparedStatementBatch(String topicName, Map>, + List> queryToRecordsMap, BlockMetaData bmd) { boolean success = false; @@ -378,6 +381,7 @@ public void addToPreparedStatementBatch(Map recordsList = entry.getValue(); for (ClickHouseStruct record : recordsList) { + bmd.update(record); //List fields = record.getStruct().schema().fields(); //ToDO: @@ -424,10 +428,14 @@ public void addToPreparedStatementBatch(Map partitionToOffsetMap = writer.groupQueryWithRecords(records, queryToRecordsMap); + BlockMetaData bmd = new BlockMetaData(); - if(flushRecordsToClickHouse(writer, queryToRecordsMap)) { + if(flushRecordsToClickHouse(topicName, writer, queryToRecordsMap, bmd)) { // Remove the entry. queryToRecordsMap.remove(topicName); } @@ -205,7 +208,8 @@ private void processRecordsByTopic(String topicName, ConcurrentLinkedQueue>, List> queryToRecordsMap) { + private boolean flushRecordsToClickHouse(String topicName, DbWriter writer, Map>, + List> queryToRecordsMap, BlockMetaData bmd) { boolean result = false; @@ -213,7 +217,13 @@ private boolean flushRecordsToClickHouse(DbWriter writer, Map this.transactionId) { + this.transactionId = gtId; + } + } + if(record.getPos() > binLogPosition) { + this.binLogPosition = record.getPos(); + } + + this.partition = record.getKafkaPartition(); + + + this.topicName = record.getTopic(); + } } From 9c6b3766674e531c54deb5efce2242cbb2e37738 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Wed, 27 Jul 2022 11:01:52 -0400 Subject: [PATCH 06/13] Added partition to offset metrics(Prometheus) --- deploy/docker/docker-compose.yaml | 19 ++ deploy/sysbench/compare_mysql_ch.sh | 2 +- doc/Performance.md | 8 +- docker/grafana/Dockerfile | 4 +- .../config/altinity_sink_connector.json | 167 +++++++++++++++++- .../sink/connector/common/Metrics.java | 17 ++ .../sink/connector/model/BlockMetaData.java | 25 ++- 7 files changed, 231 insertions(+), 11 deletions(-) diff --git a/deploy/docker/docker-compose.yaml b/deploy/docker/docker-compose.yaml index 625116d17..fedfdddcf 100644 --- a/deploy/docker/docker-compose.yaml +++ b/deploy/docker/docker-compose.yaml @@ -181,6 +181,25 @@ services: # KAFKA_BROKERS: kafka:19092 # restart: unless-stopped + console: + image: docker.redpanda.com/vectorized/console:master-217260f + restart: on-failure + entrypoint: /bin/sh + command: -c "echo \"$$CONSOLE_CONFIG_FILE\" > /tmp/config.yml; /app/console" + environment: + CONFIG_FILEPATH: /tmp/config.yml + CONSOLE_CONFIG_FILE: | + kafka: + brokers: ["kafka:9092"] + connect: + enabled: true + clusters: + - name: datagen + url: http://debezium:8083 + ports: + - "8089:8080" + depends_on: + - kafka sink: diff --git a/deploy/sysbench/compare_mysql_ch.sh b/deploy/sysbench/compare_mysql_ch.sh index 34da2f956..0676be091 100755 --- a/deploy/sysbench/compare_mysql_ch.sh +++ b/deploy/sysbench/compare_mysql_ch.sh @@ -4,7 +4,7 @@ rm -fr MySQL.tsv rm -fr CH.tsv -docker exec -it clickhouse clickhouse-client -uroot --password root --query "select id ,k ,c , pad from test.sbtest1 where sign !=-1 order by id format TSV" | grep -v "" >CH.tsv +docker exec -it clickhouse clickhouse-client -uroot --password root --query "select id ,k from test.sbtest1 where sign !=-1 order by id format TSV" | grep -v "" >CH.tsv docker exec -it mysql-master mysql -uroot -proot -B -N -e "select * from sbtest.sbtest1 order by id" | grep -v "Using a password on the command line interface" >MySQL.tsv diff --strip-trailing-cr MySQL.tsv CH.tsv \ No newline at end of file diff --git a/doc/Performance.md b/doc/Performance.md index 5f9c696fa..0275cdf55 100644 --- a/doc/Performance.md +++ b/doc/Performance.md @@ -50,4 +50,10 @@ group by database, table, event_type, partition_id order by c desc` Target: -5 threads , 600k/second \ No newline at end of file +5 threads , 600k/second + +## Binary logs + +`show binary logs;` + +`show binlog events in `mysql-bin.000003` \ No newline at end of file diff --git a/docker/grafana/Dockerfile b/docker/grafana/Dockerfile index 5cde5ca98..7bf2bad88 100644 --- a/docker/grafana/Dockerfile +++ b/docker/grafana/Dockerfile @@ -4,5 +4,5 @@ FROM grafana/grafana:${GRAFANA_VERSION} COPY ./config/dashboard.yml /etc/grafana/provisioning/dashboards COPY ./config/datasource.yml /etc/grafana/provisioning/datasources COPY ./config/debezium-dashboard.json /var/lib/grafana/dashboards/debezium-dashboard.json -COPY ./config/altinity_sink_connector.json /var/lib/grafana/dashboards/altinity_sink_connector.json -COPY ./config/debezium-mysql-connector.json /var/lib/grafana/dashboards/debezium-mysql-connector.json +COPY ./config/altinity_sink_connector.json /etc/grafana/provisioning/dashboards +COPY ./config/debezium-mysql-connector.json /etc/grafana/provisioning/dashboards diff --git a/docker/grafana/config/altinity_sink_connector.json b/docker/grafana/config/altinity_sink_connector.json index 4bc3e8a22..a709c0580 100644 --- a/docker/grafana/config/altinity_sink_connector.json +++ b/docker/grafana/config/altinity_sink_connector.json @@ -228,8 +228,8 @@ ] }, "gridPos": { - "h": 5, - "w": 22, + "h": 8, + "w": 10, "x": 0, "y": 5 }, @@ -253,6 +253,169 @@ ], "title": "Records (Topic)", "type": "table" + }, + { + "datasource": "prometheus", + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "stepAfter", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 10, + "y": 5 + }, + "id": 10, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "exemplar": true, + "expr": "clickhouse_topics_num_records_total{}", + "interval": "", + "legendFormat": "", + "refId": "A" + } + ], + "title": "Total Records", + "type": "timeseries" + }, + { + "datasource": "prometheus", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 13 + }, + "id": 12, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "exemplar": true, + "expr": "clickhouse_sink_partition_offset{}", + "interval": "", + "legendFormat": "", + "refId": "A" + } + ], + "title": "Kafka Offset", + "type": "timeseries" } ], "refresh": false, diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/common/Metrics.java b/src/main/java/com/altinity/clickhouse/sink/connector/common/Metrics.java index 4b535a893..ddaa5c228 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/common/Metrics.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/common/Metrics.java @@ -52,6 +52,8 @@ public class Metrics { private static Gauge maxBinLogPositionCounter; + private static Gauge partitionOffsetCounter; + private static Gauge gtidCounter; private static HttpServer server; @@ -116,6 +118,10 @@ private static void registerMetrics(CollectorRegistry collectorRegistry) { maxBinLogPositionCounter = Gauge.build().name("clickhouse_sink_binlog_pos").help("Bin Log Position").register(collectorRegistry); gtidCounter = Gauge.build().name("clickhouse_sink_gtid").help("GTID Transaction Id").register(collectorRegistry); + partitionOffsetCounter = Gauge.build(). + labelNames("Topic", "Partition"). + name("clickhouse_sink_partition_offset").help("Kafka partition Offset").register(collectorRegistry); + topicsNumRecordsCounter = Counter.builder("clickhouse.topics.num.records"); } @@ -158,6 +164,17 @@ public static void updateMetrics(BlockMetaData bmd) { gtidCounter.set(bmd.getTransactionId()); //tag("partition", Integer.toString(bmd.getPartition())). //register(Metrics.meterRegistry()).increment(bmd.getTransactionId()); + + HashMap> partitionToOffsetMap = bmd.getPartitionToOffsetMap(); + + if(!partitionToOffsetMap.isEmpty()) { + for(Map.Entry> entry : partitionToOffsetMap.entrySet()) { + MutablePair mp = entry.getValue(); + partitionOffsetCounter.labels(entry.getKey(), Integer.toString(mp.left)) + .set(mp.right); + } + + } } public static void updateSinkRecordsCounter(String blockUUid, Long taskId, String topicName, String tableName, diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/model/BlockMetaData.java b/src/main/java/com/altinity/clickhouse/sink/connector/model/BlockMetaData.java index ea6321f50..395b4e14f 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/model/BlockMetaData.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/model/BlockMetaData.java @@ -16,7 +16,7 @@ public class BlockMetaData { // Map of partitions to offsets. @Getter @Setter - HashMap> partitionToOffsetMap; + HashMap> partitionToOffsetMap = new HashMap<>(); @Getter @Setter @@ -72,19 +72,34 @@ public class BlockMetaData { public void update(ClickHouseStruct record) { int gtId = record.getGtid(); - if(gtId != -1) { - if(gtId > this.transactionId) { + if (gtId != -1) { + if (gtId > this.transactionId) { this.transactionId = gtId; } } - if(record.getPos() > binLogPosition) { + if (record.getPos() > binLogPosition) { this.binLogPosition = record.getPos(); } this.partition = record.getKafkaPartition(); - + long offset = record.getKafkaOffset(); this.topicName = record.getTopic(); + if (partitionToOffsetMap.containsKey(this.topicName)) { + MutablePair mp = partitionToOffsetMap.get(this.topicName); + if (offset >= mp.right) { + // Update ap. + mp.right = offset; + mp.left = partition; + partitionToOffsetMap.put(topicName, mp); + } + } else { + MutablePair mp = new MutablePair<>(); + mp.right = offset; + mp.left = partition; + partitionToOffsetMap.put(topicName, mp); + } } + } From f13f0c9836f5a7ab6bfc93954bc4e383ffe769a9 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sun, 31 Jul 2022 10:34:50 -0400 Subject: [PATCH 07/13] Confluent schema registry changes. --- ...ebezium-connector-setup-schema-registry.sh | 125 ++++++--- ...ker-compose-confluent-schema-registry.yaml | 247 ++++++++++++++++++ .../sink-connector-setup-schema-registry.sh | 141 ++++++---- docker/Dockerfile-sink-on-debezium-base-image | 23 +- docker/Dockerfile-source-debezium-base-image | 75 ++++++ docker/build-source-on-debezium-base.sh | 29 ++ 6 files changed, 550 insertions(+), 90 deletions(-) create mode 100755 deploy/docker/docker-compose-confluent-schema-registry.yaml create mode 100644 docker/Dockerfile-source-debezium-base-image create mode 100755 docker/build-source-on-debezium-base.sh diff --git a/deploy/debezium-connector-setup-schema-registry.sh b/deploy/debezium-connector-setup-schema-registry.sh index daa427b44..0f9d29808 100755 --- a/deploy/debezium-connector-setup-schema-registry.sh +++ b/deploy/debezium-connector-setup-schema-registry.sh @@ -62,48 +62,91 @@ fi # "value.converter":"io.confluent.connect.avro.AvroConverter", # "value.converter.schema.registry.url":"http://schemaregistry:8081" +if [[ $2 == "apicurio" ]]; then + echo "APICURIO SCHEMA REGISTRY" + #https://debezium.io/documentation/reference/stable/configuration/avro.html + cat </dev/null 2>&1 && pwd)" +SRC_ROOT="$(realpath "${CUR_DIR}/..")" + +# Externally configurable build-dependent options +TAG=$(date +%F) +#TAG="${TAG:-latest}" +DOCKER_IMAGE="altinity/source-connector-on-debezium-base:${TAG}" + +# Externally configurable build-dependent options +DOCKERFILE_DIR="${SRC_ROOT}/docker" +DOCKERFILE="${DOCKERFILE_DIR}/Dockerfile-source-debezium-base-image" + +echo "***************" +echo "* Build image *" +echo "***************" +DOCKER_CMD="docker build -t ${DOCKER_IMAGE} -f ${DOCKERFILE} ${SRC_ROOT}" + +if ${DOCKER_CMD}; then + echo "ALL DONE" +else + echo "FAILED" + exit 1 +fi From a30686c34af54294b88763800c3297e2c9a98417 Mon Sep 17 00:00:00 2001 From: Kanthi Date: Sun, 31 Jul 2022 16:44:28 -0400 Subject: [PATCH 08/13] Revert "Map INT32 Kafka connect to INT256 CH type" --- .../sink/connector/converters/ClickHouseDataTypeMapper.java | 3 +-- .../connector/converters/ClickHouseDataTypeMapperTest.java | 3 --- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/converters/ClickHouseDataTypeMapper.java b/src/main/java/com/altinity/clickhouse/sink/connector/converters/ClickHouseDataTypeMapper.java index 9181248bb..4e484f5c5 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/converters/ClickHouseDataTypeMapper.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/converters/ClickHouseDataTypeMapper.java @@ -25,8 +25,7 @@ public class ClickHouseDataTypeMapper { dataTypesMap.put(new MutablePair(Schema.INT16_SCHEMA.type(), null), ClickHouseDataType.Int16); dataTypesMap.put(new MutablePair(Schema.INT8_SCHEMA.type(), null), ClickHouseDataType.Int8); dataTypesMap.put(new MutablePair(Schema.INT32_SCHEMA.type(), null), ClickHouseDataType.Int32); - - dataTypesMap.put(new MutablePair(Schema.INT64_SCHEMA.type(), null), ClickHouseDataType.Int256); + dataTypesMap.put(new MutablePair(Schema.INT64_SCHEMA.type(), null), ClickHouseDataType.Int64); // Float dataTypesMap.put(new MutablePair(Schema.FLOAT32_SCHEMA.type(), null), ClickHouseDataType.Float32); diff --git a/src/test/com/altinity/clickhouse/sink/connector/converters/ClickHouseDataTypeMapperTest.java b/src/test/com/altinity/clickhouse/sink/connector/converters/ClickHouseDataTypeMapperTest.java index 69d9298fa..464b58cd7 100644 --- a/src/test/com/altinity/clickhouse/sink/connector/converters/ClickHouseDataTypeMapperTest.java +++ b/src/test/com/altinity/clickhouse/sink/connector/converters/ClickHouseDataTypeMapperTest.java @@ -26,9 +26,6 @@ public void getClickHouseDataType() { chDataType = ClickHouseDataTypeMapper.getClickHouseDataType(Schema.INT32_SCHEMA.type(), Date.SCHEMA_NAME); Assert.assertTrue(chDataType.name().equalsIgnoreCase("Date32")); - chDataType = ClickHouseDataTypeMapper.getClickHouseDataType(Schema.INT64_SCHEMA.type(), null); - Assert.assertTrue(chDataType.name().equalsIgnoreCase("INT256")); - } } From 1d3e5840583a489b5713cf108292cca213563fb5 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Mon, 1 Aug 2022 15:09:53 -0400 Subject: [PATCH 09/13] Fixed grafana dashboard auto loading. --- deploy/debezium-connector-setup-sysbench.sh | 111 +++++++++----- ...ker-compose-confluent-schema-registry.yaml | 78 ++++++++-- deploy/docker/docker-compose.yaml | 1 + deploy/sink-connector-setup-sysbench.sh | 141 ++++++++++++------ docker/grafana/Dockerfile | 4 +- 5 files changed, 237 insertions(+), 98 deletions(-) diff --git a/deploy/debezium-connector-setup-sysbench.sh b/deploy/debezium-connector-setup-sysbench.sh index 845582376..69560c0c8 100755 --- a/deploy/debezium-connector-setup-sysbench.sh +++ b/deploy/debezium-connector-setup-sysbench.sh @@ -27,47 +27,88 @@ DATABASE_SERVER_ID="5432" # Alphanumeric characters, hyphens, dots and underscores only. DATABASE_SERVER_NAME="SERVER5432" +if [[ $2 == "apicurio" ]]; then + echo "APICURIO SCHEMA REGISTRY" + ###### Connector for sysbench test ###### + cat < /tmp/config.yml; /app/console" + environment: + CONFIG_FILEPATH: /tmp/config.yml + CONSOLE_CONFIG_FILE: | + kafka: + brokers: ["kafka:9092"] + connect: + enabled: true + clusters: + - name: datagen + url: http://debezium:8083 + ports: + - "8089:8080" + depends_on: + - kafka sink: @@ -209,13 +253,13 @@ services: SERVICE_PORT: 9072 volumes: - ../config/jmx-config.yml:/opt/jmx_exporter/config.yml - - blackbox_exporter: - container_name: blackbox_exporter - image: prom/blackbox-exporter:master - restart: "no" - ports: - - "9115:9115" +# +# blackbox_exporter: +# container_name: blackbox_exporter +# image: prom/blackbox-exporter:master +# restart: "no" +# ports: +# - "9115:9115" prometheus: container_name: prometheus @@ -227,14 +271,22 @@ services: - ../config/prometheus.yml:/opt/bitnami/prometheus/conf/prometheus.yml grafana: - container_name: grafana - image: grafana/grafana + build: + context: ../../docker/grafana + args: + GRAFANA_VERSION: latest + #container_name: grafana + #image: grafana/grafana restart: "no" - volumes: - - ../config/grafana/dashboards:/etc/grafana/provisioning/dashboards + #volumes: + # - ../config/grafana/dashboards:/etc/grafana/provisioning/dashboards ports: - "3000:3000" - + links: + - prometheus + environment: + - DS_PROMETHEUS=prometheus + - GF_USERS_DEFAULT_THEME=light #### END OF MONITORING ### bash: diff --git a/deploy/docker/docker-compose.yaml b/deploy/docker/docker-compose.yaml index fedfdddcf..a6848b220 100644 --- a/deploy/docker/docker-compose.yaml +++ b/deploy/docker/docker-compose.yaml @@ -295,6 +295,7 @@ services: - prometheus environment: - DS_PROMETHEUS=prometheus + - GF_USERS_DEFAULT_THEME=light #### END OF MONITORING ### bash: diff --git a/deploy/sink-connector-setup-sysbench.sh b/deploy/sink-connector-setup-sysbench.sh index 53b12216b..23f7cd20e 100755 --- a/deploy/sink-connector-setup-sysbench.sh +++ b/deploy/sink-connector-setup-sysbench.sh @@ -24,54 +24,99 @@ TOPICS_TABLE_MAP="SERVER5432.test.employees_predated:employees, SERVER5432.test. #"topics": "${TOPICS}", -cat < Date: Tue, 2 Aug 2022 10:51:13 -0400 Subject: [PATCH 10/13] Support for JSON data type. Both MySQL/Postgres JSON types map to String with io.debezium.data.JSON name --- deploy/docker/docker-compose-postgresql.yaml | 14 +++++++------- deploy/sink-connector-setup-schema-registry.sh | 15 ++++++++++----- deploy/sql/init_clickhouse.sql | 5 ++++- deploy/sql/init_postgres.sql | 2 ++ pom.xml | 2 +- .../converters/ClickHouseDataTypeMapper.java | 3 +++ .../clickhouse/sink/connector/db/DbWriter.java | 15 ++++++++++++--- .../sink/connector/model/BlockMetaData.java | 11 ++++++++--- .../sink/connector/model/ClickHouseStruct.java | 4 ++-- 9 files changed, 49 insertions(+), 22 deletions(-) diff --git a/deploy/docker/docker-compose-postgresql.yaml b/deploy/docker/docker-compose-postgresql.yaml index a0edc8f34..6e84d86ac 100644 --- a/deploy/docker/docker-compose-postgresql.yaml +++ b/deploy/docker/docker-compose-postgresql.yaml @@ -108,7 +108,7 @@ services: sink: container_name: sink - image: altinity/clickhouse-sink-connector:latest + image: altinity/clickhouse-sink-connector:${SINK_VERSION} restart: "no" ports: - "18083:8083" @@ -167,12 +167,12 @@ services: volumes: - ../config/jmx-config.yml:/opt/jmx_exporter/config.yml - blackbox_exporter: - container_name: blackbox_exporter - image: prom/blackbox-exporter:master - restart: "no" - ports: - - "9115:9115" +# blackbox_exporter: +# container_name: blackbox_exporter +# image: prom/blackbox-exporter:master +# restart: "no" +# ports: +# - "9115:9115" prometheus: container_name: prometheus diff --git a/deploy/sink-connector-setup-schema-registry.sh b/deploy/sink-connector-setup-schema-registry.sh index 196a0f960..b68e26f76 100755 --- a/deploy/sink-connector-setup-schema-registry.sh +++ b/deploy/sink-connector-setup-schema-registry.sh @@ -16,15 +16,19 @@ CLICKHOUSE_DATABASE="test" BUFFER_COUNT=10000 #SERVER5432.transaction -TOPICS="SERVER5432.test.employees_predated, SERVER5432.test.customers" -TOPICS_TABLE_MAP="SERVER5432.test.employees_predated:employees, SERVER5432.test.products:products" +if [[ $1 == "postgres" ]]; then + TOPICS="SERVER5432.public.Employee" +else + TOPICS="SERVER5432.test.employees_predated, SERVER5432.test.customers" + TOPICS_TABLE_MAP="SERVER5432.test.employees_predated:employees, SERVER5432.test.products:products" +fi #TOPICS="SERVER5432" #"topics.regex": "SERVER5432.sbtest.(.*), SERVER5432.test.(.*)", #"topics": "${TOPICS}", -if [[ $1 == "apicurio" ]]; then +if [[ $2 == "apicurio" ]]; then echo "APICURIO SCHEMA REGISTRY" cat <com.clickhouse clickhouse-jdbc - 0.3.2-patch8 + 0.3.2-patch11 http diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/converters/ClickHouseDataTypeMapper.java b/src/main/java/com/altinity/clickhouse/sink/connector/converters/ClickHouseDataTypeMapper.java index 4e484f5c5..04c675c7c 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/converters/ClickHouseDataTypeMapper.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/converters/ClickHouseDataTypeMapper.java @@ -2,6 +2,7 @@ import com.clickhouse.client.ClickHouseDataType; import io.debezium.data.Enum; +import io.debezium.data.Json; import io.debezium.time.*; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.kafka.connect.data.Decimal; @@ -61,6 +62,8 @@ public class ClickHouseDataTypeMapper { dataTypesMap.put(new MutablePair<>(Schema.Type.STRING, ZonedTimestamp.SCHEMA_NAME), ClickHouseDataType.String); dataTypesMap.put(new MutablePair<>(Schema.Type.STRING, Enum.LOGICAL_NAME), ClickHouseDataType.String); + + dataTypesMap.put(new MutablePair<>(Schema.Type.STRING, Json.LOGICAL_NAME), ClickHouseDataType.JSON); } public static ClickHouseDataType getClickHouseDataType(Schema.Type kafkaConnectType, String schemaName) { diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java b/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java index d209cd0c2..374277b07 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java @@ -15,6 +15,7 @@ import com.clickhouse.client.ClickHouseCredentials; import com.clickhouse.client.ClickHouseNode; import com.google.common.io.BaseEncoding; +import io.debezium.data.Json; import io.debezium.time.Date; import io.debezium.time.*; import org.apache.commons.lang3.tuple.MutablePair; @@ -381,7 +382,11 @@ public BlockMetaData addToPreparedStatementBatch(String topicName, Map recordsList = entry.getValue(); for (ClickHouseStruct record : recordsList) { - bmd.update(record); + try { + bmd.update(record); + } catch(Exception e) { + log.error("**** ERROR: updating Prometheus", e); + } //List fields = record.getStruct().schema().fields(); //ToDO: @@ -569,7 +574,10 @@ public void insertPreparedStatement(Map columnNameToIndexMap, P // MySQL(Timestamp) -> String, name(ZonedTimestamp) -> Clickhouse(DateTime) ps.setString(index, DebeziumConverter.ZonedTimestampConverter.convert(value)); - } else { + } else if(schemaName != null && schemaName.equalsIgnoreCase(Json.LOGICAL_NAME)) { + // if the column is JSON, it should be written, String otherwise + ps.setObject(index, value); + }else { ps.setString(index, (String) value); } } else if (isFieldTypeInt) { @@ -617,7 +625,8 @@ else if (value instanceof Long) { ps.setString(index, BaseEncoding.base16().lowerCase().encode(((ByteBuffer) value).array())); } - } else { + } + else { log.error("Data Type not supported: {}", colName); } diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/model/BlockMetaData.java b/src/main/java/com/altinity/clickhouse/sink/connector/model/BlockMetaData.java index 395b4e14f..a5045060a 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/model/BlockMetaData.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/model/BlockMetaData.java @@ -77,14 +77,19 @@ public void update(ClickHouseStruct record) { this.transactionId = gtId; } } - if (record.getPos() > binLogPosition) { + if (record.getPos() != null && record.getPos() > binLogPosition) { this.binLogPosition = record.getPos(); } - this.partition = record.getKafkaPartition(); + if(record.getKafkaPartition() != null) { + this.partition = record.getKafkaPartition(); + } + long offset = record.getKafkaOffset(); - this.topicName = record.getTopic(); + if(record.getTopic() != null) { + this.topicName = record.getTopic(); + } if (partitionToOffsetMap.containsKey(this.topicName)) { MutablePair mp = partitionToOffsetMap.get(this.topicName); diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/model/ClickHouseStruct.java b/src/main/java/com/altinity/clickhouse/sink/connector/model/ClickHouseStruct.java index 5f77a7abe..5ddeb1bde 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/model/ClickHouseStruct.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/model/ClickHouseStruct.java @@ -57,11 +57,11 @@ public class ClickHouseStruct { @Getter @Setter - private String file; + private String file = ""; @Getter @Setter - private Long pos; + private Long pos = 0L; @Getter @Setter From 0f851a1996c069d7a81e4377f667acdaf80f37a8 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Tue, 2 Aug 2022 17:41:38 -0400 Subject: [PATCH 11/13] Fixed logic of retry for auto creating table --- README.md | 7 ++- ...ebezium-connector-setup-schema-registry.sh | 4 +- .../sink-connector-setup-schema-registry.sh | 2 +- deploy/sink-connector-setup-sysbench.sh | 2 +- deploy/sysbench/compare_mysql_ch.sh | 9 +++- doc/img/Grafana_dashboard.png | Bin 0 -> 119442 bytes doc/img/replacingmergetree_update_delete.jpg | Bin 0 -> 45464 bytes doc/img/sink_connector_mysql_architecture.jpg | Bin 0 -> 43478 bytes doc/mutable_data.md | 4 +- .../sink/connector/db/DbWriter.java | 40 +++++++++++++----- .../sink/connector/db/QueryFormatter.java | 9 +++- .../executor/ClickHouseBatchRunnable.java | 26 ++++++++---- 12 files changed, 73 insertions(+), 30 deletions(-) create mode 100644 doc/img/Grafana_dashboard.png create mode 100644 doc/img/replacingmergetree_update_delete.jpg create mode 100644 doc/img/sink_connector_mysql_architecture.jpg diff --git a/README.md b/README.md index bd4ab534a..f7f48ec87 100644 --- a/README.md +++ b/README.md @@ -3,8 +3,9 @@ Sink connector sinks data from Kafka into Clickhouse. The connector is tested with the following converters - JsonConverter -- AvroConverter (Using [Apicurio Schema Registry](https://www.apicur.io/registry/)) +- AvroConverter (Using [Apicurio Schema Registry](https://www.apicur.io/registry/) and Confluent Schema Registry) +![](doc/img/sink_connector_mysql_architecture.jpg) # Features - Inserts, Updates and Deletes using ReplacingMergeTree/CollapsingMergeTree - [Updates/Deletes](doc/mutable_data.md) - Deduplication logic to dedupe records from Kafka topic.(Based on Primary Key) @@ -17,6 +18,10 @@ The connector is tested with the following converters - Kafka Offset management in ClickHouse - Increased Parallelism(Customize thread pool for JDBC connections) + + +### Grafana Dashboard +![](doc/img/Grafana_dashboard.png) \ # Source Databases - MySQL (Debezium) - PostgreSQL (Debezium) (Testing in progress) diff --git a/deploy/debezium-connector-setup-schema-registry.sh b/deploy/debezium-connector-setup-schema-registry.sh index 0f9d29808..53779ee2d 100755 --- a/deploy/debezium-connector-setup-schema-registry.sh +++ b/deploy/debezium-connector-setup-schema-registry.sh @@ -138,9 +138,9 @@ else "key.converter.schema.registry.url": "http://schemaregistry:8081", "value.converter.schema.registry.url":"http://schemaregistry:8081", - "topic.creation.$alias.partitions": 6, + "topic.creation.$alias.partitions": 1, "topic.creation.default.replication.factor": 1, - "topic.creation.default.partitions": 6, + "topic.creation.default.partitions": 1, "provide.transaction.metadata": "true" } diff --git a/deploy/sink-connector-setup-schema-registry.sh b/deploy/sink-connector-setup-schema-registry.sh index b68e26f76..c67a89303 100755 --- a/deploy/sink-connector-setup-schema-registry.sh +++ b/deploy/sink-connector-setup-schema-registry.sh @@ -115,7 +115,7 @@ else "replacingmergetree.delete.column": "sign", - "auto.create.tables": false, + "auto.create.tables": true, "schema.evolution": false, "deduplication.policy": "off" diff --git a/deploy/sink-connector-setup-sysbench.sh b/deploy/sink-connector-setup-sysbench.sh index 23f7cd20e..ed61dec4c 100755 --- a/deploy/sink-connector-setup-sysbench.sh +++ b/deploy/sink-connector-setup-sysbench.sh @@ -95,7 +95,7 @@ else "value.converter.schema.registry.url":"http://schemaregistry:8081", "store.kafka.metadata": true, - "topic.creation.default.partitions": 6, + "topic.creation.default.partitions": 1, "store.raw.data": false, "store.raw.data.column": "raw_data", diff --git a/deploy/sysbench/compare_mysql_ch.sh b/deploy/sysbench/compare_mysql_ch.sh index 0676be091..2f9b4b617 100755 --- a/deploy/sysbench/compare_mysql_ch.sh +++ b/deploy/sysbench/compare_mysql_ch.sh @@ -4,7 +4,12 @@ rm -fr MySQL.tsv rm -fr CH.tsv -docker exec -it clickhouse clickhouse-client -uroot --password root --query "select id ,k from test.sbtest1 where sign !=-1 order by id format TSV" | grep -v "" >CH.tsv -docker exec -it mysql-master mysql -uroot -proot -B -N -e "select * from sbtest.sbtest1 order by id" | grep -v "Using a password on the command line interface" >MySQL.tsv +if [[ $1 == "bulk_insert" ]]; then + docker exec -it clickhouse clickhouse-client -uroot --password root --query "select id ,k from test.sbtest1 where sign !=-1 order by id format TSV" | grep -v "" >CH.tsv +else + docker exec -it clickhouse clickhouse-client -uroot --password root --query "select id ,k, c, pad from test.sbtest1 where sign !=-1 order by id format TSV" | grep -v "" >CH.tsv + +fi + docker exec -it mysql-master mysql -uroot -proot -B -N -e "select * from sbtest.sbtest1 order by id" | grep -v "Using a password on the command line interface" >MySQL.tsv diff --strip-trailing-cr MySQL.tsv CH.tsv \ No newline at end of file diff --git a/doc/img/Grafana_dashboard.png b/doc/img/Grafana_dashboard.png new file mode 100644 index 0000000000000000000000000000000000000000..e63066472ab2001ca5a0498469d3fde89f7b237b GIT binary patch literal 119442 zcmcG#byytBw=N73T!OoMaCZp7B@i5fyEAA8CqRM(hv1e#u;A`4gHM3qu7l3tgL7x^ zea`vqbH9J@SI;xuPuEmeSFcsIYOQy@9ra#a5gUUX0|^NUTUkj?3keD34HD9GQ#2Gr zjmlo{A>#DH^R2QD8lnWCSw|wS$-LwZytDzfUO)>E8zehdfQt>MrJO9r^`~63Y!Ftp9s-|BoMT>p=CN$G3}GSXg}iEHP*=P~^>e z%14T2U|87K+e<&`0T>NpHBD?v``5cX(b3VxyAP$~sgCwjbWf|;+mj*#Vay9}o*Iup zZIrM(nBCwXxiB>4#kQcnJZHlkt@hkX)Bk?A#HUvUlj*y$Cs8%^^-6kGVNEvDt2;g^ z_X)w`CO!Wz1y=W`gpr3Qfrx#szpdOVq7kXwxFu6DM8-YUf2QKyY`Jj7^BT}`F zA9ttJjX6E5R}RFW;eq4ya}6Ld*6U1jGOIsyetIV3#cRRuT4KU=D(>F-EL=Cm0J?je z`7`EUgPda2wT#wb{0^eSsiIF#eRtt$Z8*td@E?zBlda>v;V$F~Av!I6kNy`2;;)`7 z5RYXg&;IBq6?Npu5(32DbZ8A=sNFbPk}EL&E#x!kt^31Z`jhZUWma~EC7lB9kPB)>xo^HV4)C>;O{4rH$ zrWk(z)U)G*!0d~Ti3c>FwNf6NGNGU2kdz_Eo9p#Ce>yE(geZ&V+f_V_8a_7g}M z9_E{mI0~i8WKxUu<0dm+kN?`r?%m35P;zlZ5o0ukXPqRSs!ml>8-W0txqmow*eYa| ze@D;#XqTYLOU^zK#XEn3z%CsIKoCZRL<$izLI5!PIhI2BeC7{Xwsdevz|C2ax6eOQ z5caJO5@6UABs&JHtmN7T2KslN2A})}bC|V|LnuWplTb90sd{>Pu1z{dK1U^!@s&Io zCBJCdgn$rN=r}lWqN!VhG20V4NJZf60YsgM{{)GX(_s@mlszbj!=O1Gaq>xJk1j4o z70vOKF#MPjhKgI``hbEE-_O9dS+*EBT~^V>7V6q2R?c)(JK$cmSlE!)QDli*allUy zIM5m6pu3*p1YePg+Bf2~mu_bn;4jVL6O?$@|1i3Xgl-8gpQxw4<}9WW*;@I<@X*#! zbb$(=Z?tE1Y310}vi|@|zv^{;>W4CHe5=%Em`S2RN~8;}aUcFzi9%sz%AYQ#-DM)_ zVNg_F316W&bN_M_HT%Ha?tGtAf7n%Nx0EREK&6mx+K#icGoI7+ilvxCMAT*@>jra5 zj1npC?i%TLjjT9gU6o#2UzY{m;r_HS_(!x)DqWO!DKv9{m>cEFgsfl!pBv$4+)r2J z1*(hgdC4M>p`PoSf}#B*HFZ0?a2)FH-#yRmC)?Rx<)CHoxbx}X-%knXMM7_QEjF!V z6c_}5>F=BoNiVN_O`KjJgKxSvw-Ie2D#-oj`X+lq&dR>xD|XnCf5i$*wke8_FFahP|Pyze`SzStm2)$B#;X>!nK00X)2Qhq26SR47XlV zoO0+3dHa8AcYTbo<^MQLR_l*ty{f+|A{E#t^k-0?#XGBuAq%7!N=az!)w34&KIvIwEe%MapP2B>FI}gTvtNBh<3>oV z8_-9EP5i$ttZ&-ULR(%PWCA&1Z6*bUn?sm<>q+bISkluc;20O_414)hOja$ap}~9X zslY!#?HR3z$csxwMEm{inn<1yM7VZ#Zo*>UViWeF*;?p6#a{WZZ?^ zRi;nIXaA-=S9%F}^vry39T{mG==}o-hN`8DL?M*=Jp}k55G7og-%I21l$e-ZcraD( zUa3FBzzw>-(tZM zuZF=>0_?CG#zo>_)oEh7jOPTAlKJOf4kdIM@ZUM<0i*|F%{{ZFS!I&r0v(P{1w`G_ zUYc|m`j)ZPv5;6l)cZO=T+*W~W&8YrzFBikv2tNAy^Ya~*fS17!Yzv{R7}jfaeOpY zTM4hbkA9v0+@dL-FL=)%)kMp2CHq&GIryky%K(5%83AWE1{gB2knx zCh_s;gIE^Rp@zGp2%)woi^O4lzR}%?_19?%`WQ|EX{+1LHVG_@f?x*jDZ_26?qxQe zkz3DlQGv$=-cV%FFW$mo*`jwQJ(Heu=={L^uR!5juPfSzSujs)q|R)3As@!QJL z5kdKXUogJ4g7|?_kUr(z|4gBDB5C>NTPt1+> z)6lFPGmutyhw`3>YyIKfuM`GXeu;je_#uHaSbp*9NREi4Eu9Y!ccMDrN+L+YE&R@6 zJz<3k25z0K_xqU3{Q0j?YJ z3mOyXsn%e#M4=V2!6W#zL3GUV#6v^f&Et^Z*$fRR&E>p!gg4MM!1L4c@vjcCobxe9MSbP0YuXC-x5xMHvsR|oS zZ*gPjG70`v+Cot1)ReCQ8R1^mCLksyjjV$C#@%IaYnsz`c6MIhns(35hO4B!Hg|Fw z6+S27v%D~<6|?CX7u%VEeD{)Dh_*6QxB#8Dd ziUs^d!&694Pp`SPd3|3KM_-?0FqoHZJef9wlFHrNePg-h{EZi%^}xssrL=p}`nvgu z|F;6)I%WeSBhO_B3p~IY$J6- z$K_9kh9tjL%x!W;N6~51xt$tRA6c~wQ@68Pua5^_U#}F};(Yb#2vC_1sk%6R{#x>4!RV!dQpLd}4!_UPV zU+sz9^Bwg!NFABN1B;7d-UPJA;vZwrGjnn{lb)4@F90>kjLr|5!Vw@sLMwn zhHbR}v_KvAKmu8zWd||QBZ;TKx7De#_GnbnTUTYPr&%zVpjN3s*j3D2vXyi z=KN9Usg2F|(`3n{R>G@LQ|YhPs$P1dRQynA>hLPBi&A$gKATKd`^2lx{maeu>49N7 zm8f{QdLRaDtK2=XqDC$dz7ONJ><4i}t=@V^;6Zt_j+X*U)8IGZFxsF6&+O2dPIbN; zJiKenEx(DK2fgG40VZ6%&!=-eU$AIqNsyO_Y3aqFpiqU3{Re$eA7Kp5koPbCrZ5byUve!o$C;EY3ZrM z&s;^SSP926ezL+ei-m&KI)4Bv200d{(hKUOA;A`587p@SH<&FKRFbVvk^YcG=j`pT zL^9b6g3+2C{^vdi0S6Mkp1cYsw&Q1Y0P#{{ zy@dSptopWc(xtqDD@n@(jOOMnW!XFa6j7`|j}_BZ@wm=9*?p;9BU@rMjs7rx4l?nt z7TTM4ZSiZoX1?g%_IL2|_|-#-86^?km^X zzSF8ULl+NzeAaP?(>*ak%v~i$DIF9l6^Qz5QaLV;YH{EKGN# zlhne(JSR3IP_95aI zvp{uo0rLIdoty~I+wBC>=XErdV=CFYVR}ChzlbiMelN14LI(XEQ9$(UfBy>NSg!B(>eMN*R>tqK1;!hL$zWBs~d{}DRSZ} zqGZNMtj6#Ad%gjiK&kp=EpYb-8cc(>1Xl=Iu{3A?T z-h8)mU-TwZj5xTOTq;e=`yT#WwBAMDQ?cz;ND!Jw&p)ggg2%eBWSG|iOl?2+SkfS{d5|-e)UgLTu!*>)f>f8SNT4J zp?nSOe8J1=T;3BJP?l4<7mvM_R&jq^v&4tV-%dRBt~Gk^8A*ea&z*ZBa!vLg_FrD* zwySrvpR(|4kw0}vKPl$%7CYPyaJ*(NNTgbi$Hr{Ama!vlU_E0LqQKBQ97u3FHmILW z`?YfJ>31xkF4kycu{OfHG^STVm)zN`G0Jr=V(5$Is*xlwLu|9RaOoh5d8TKt7{Sms z-*}FHd8WW}5~KO}lly`x=TK?^Fdd`gypU4R=I6QI)Va;&@b<*aDZPtDxKCez1u*tF zXR8F3cIYw}=lm5IzLNnnc=SgvDlgUG8ZCC8v6x@)+81y_MHVz-bfa3^Y0HQnp0X!i zISe}47No$Kr5s9E3?vn@;>`7wVrs2Yprz4q2D30FOUD=?EY>Ep)?(v>@eSRD*Q@UJV(K{njQ9sIom1ylM z(-rGu4G-t}`Y#A5NUzFR0f7zug$FQ?OBEwf0$YiK#jj^~LyvbG)Qgpdc6?OB$P(w??7y=~^;_e`o(Q1Zh;ivKI_{ z9i(5ina$uzu{>Ri=&Ljdi8tLERHjSfTuVd`B*x5_nDV@upS$;7jgLeaBmYBYnR)H@ zw@f3gAG9RW7<|=N2dSOTh}nBvBvG({l&tHfjH@WYVe-UM_dG0`q9dAOU7Y&y0Q03p zT@K)F!s=!107bG1wAIVmh`|5(HdfbIJr&*`hZ(Y}(A3YVJUhL=8?J9|o+GT#(#}

`DZiBvs5SJ2awV#BiB2SvFr9|D^Xa_XX8rOfb@aS395GscXqVUJgo2XGh=TY zoS%hP-(KV0OO1|JP{4WWUaNg^%x#`09opIJA1>SH_5H#oCm}g!yH?O32VDL1u?4=K-t#u5QV>z{<_ z$1ybz7}%l97(O3D+eOtDE+90csd!0zo4lm0(bH| z7z@<=oIZ!19c~NdDF1I!Vy&c#2TpdB0(5lb#75*bD>% zl52BWd6mD!=zrS9>)NKA?w1eG-rNJw{I(p5R!)ogxm@d%A|N31P%v#aI7RR6c#)^?On(`3UF%_lWdI` zB!sF*Umw=;8lt~4GBP6OOz+#SD&qCH5c%YjOnbs{v!UB0RG^5hG28hM#6GkH>hEWq zVC9moTpL0FGnPR#0^iNXp0J5G5IzN%*+n39;|4UocMme!_?kz90JwR(+FINFWCI)# zSc~%8+&tjUM9wEUl}_ItnI6*E^=t;ku>`BCV{GShHMve%rQu%heEcdM0N<r6{&8ET6-KpwBL-r={LVvE%(#I5 zvl!QJ9J--rDX%wj#UF2IwTOdB+jg7GpYC})^hg!UUJt`c&12f5pN_Ez$B3J{KC~Vc zmZng}XtJEPWn_$S+ma!<(Hpt42VHeNXOwWxEd25^al-a${ZjuEo;_!c3sMPB{aA0;2vji!f|G$|6pt^FBu$DNqEEtC5v-nWB{ zZ{rmIa^8ENLr=On7&11axQ!+qB~(G}M=eeLy@b8J-K5^9) z^fJyN-0id5iswanwGPv^w-*#jQTWiKrC_8_zUL(FeNW6(OHF|mXypXf;|HJhR~AdD zqyJtVLv0iN{V)9e?+mI8KMYFafLlB}b%15c*0XyzLSa?TOs}_bNPC2W?~lU(iwMw^ z!{C~;3AP+{+1k)>X~cWeQdh^1z!?6*|IVm)?AI!>p{@fbqP|}d{-TA83)M8k3-jCL zRa!(Fkz$#n3En;mli2<7)B0kSx0Tl>u?^5NjJAddGJd_&>%K*=JzW%*B~=c}hV)_w zI?RaL3!1bDf{6G{&)XM9dNh{jaUK^`tC{y?TAh*v{HT=tnXiJR{p>mJbfC5;e(772 zC8#NR6tbh{j_-)YpEt@&TP-&y-3MH=c0GKIA@T&)whckBpeA&y+v*;Yc`uMn{m-69 z?jw6cmqpLoT7RlPLtr)+zx6P8$D@45=jPH7Ih;trBi$}3Nq1ka_v~=%e+lZWo zkdyVc7odx8)jfsQhQ^;FyFna+{BH0s4ID?R(?ENMY}0w|u#D#|ru1Pc)`3y=bf$|^ zaaVmGEwoGMBnNQ@XNHqX(4>iRX@(Pje*@tgy^?lWL7kw{E;ZlmV94)%vb1fI$_CG* z%J3^Dz=yMm@Iw6e4g5i6C`rG*{wYx~2bo->r1T4JvP$Q86dJB=vrr)XK+ru#p66CM zZbUqd#c;LFn8#xbdcncgw>=+U<(lswB6eab-7{B@iEGb(E7o5swWi59Do+?=bAz5$ zL7m8E1Uy5T8rvIPy0DWxHQOIOpZFlcffK)COy8&6JqDkRa)TD?eoC8Zco)E~JHL4f z^^jUH+I^8JPuF3esm${;^aeFjg^xhjANg;9b z80r01#q5DVm$HOhY^RGjo7>tFSGfY!JYJY1a46m=-+wZN%DLD0w8X?Ud0f~Ut-f+_ zjMDfrCnki7{@Qxz=>Zl3Z~f-f0k5~^s3d$MyT8-=UG5Q=@qS#4sS@?sv-1#AU3wAe z?=HLx&KbDoW_G_(xj50HT5Pg;43)q4y`n_ectrX|HG20Qa&sUHl0HeAKt|PzNM%1t z2_M#lK0`#Wth23E2dit46I}jcg~C?Chu=v<2^3&XY{*e3Ow^r4SWxrsA_zl zE6;FLbTknK{MK!R`zxg{K44oo-Inq2u1IdBo>o7Kk2m>aSNZm=of;3aN7_o(6@jyE zrhD?;M&kZ3<7OhWH4lF`JM$ZQYDyaj7|u4CF9u_&DMRimx79BZUc;%>NNti3ci1uh z;^-?@1KVTZlcQXZbMjs0FT61RkRTXLCN*pp9-k#>v(T+RW}q7xRUxJ1H^XQkHe=g= zg6TyrnH#FgtR+?&rg72NKZ*L^T!4n{Z~Ib@!m>SrSFaQk((zGIl0U* zzGQRbgn6a0>&v>`4xOfK1t(hBqPiNxQSsG3G3pGGlJz%?(pleh^RA+NJXce1ho1-t zQn~S=3(is#@W=*FIb*X5_?K%1=}&T`r6ovQ!g!#{=^dI?#2e%qEfG&Kq?a%VR*yd z%OF%Fmf}4wXT%go4nwYLZLrz}o?TuBx|{>HN7K^su3)=^*iY%J)QBwecTdpAa)Xvk zB6fWOQc^h(j*G);x$@Wx=-m}fajTTw%5lvetiW*+N=_h>Q>Mk_;@!#HF0x0^}CJw)k?`yncXtoF){*k zLF5?~5NTamX3!r1w@O_2>ZL(b5JPS*wKB8dUpOWL^{~3)WXkWYa(6o?o*1lm$V?Um zD1Wsb4&`JKX?8~!0@iK84{;`>gp%0!VwA>eVwwk`oYGq~_n$ntX73+HU-n|fiF5#P z(#@Uh-pNIBF9prZq+rarkK?zUG6u8Z)*cy;9xR-rmn3AN+I{r6Hx+yB+g4l>A!q$M z2HB2hIMfqIHSV!9Y@m$ZP3Zdc99epOA(B(5aWIV9czlpU0Ghw4S^F^?&doWg;i!s2<_q`K4+VVXa9(v=&Yt< zQ6U@-f0_|TFHi9>Q~}y56)38acZOySiwTk4l;{Y>L8Zo`8ANYgsX*5$B*-kU{j7Jp zXCEX2Hl{FI>Qiz^zHXjsdO#?ZJ40<$-5|f!CtflINRn_Eg$ComAXAvZVLc>z(||Dr zsFcYJVJ|}-c8`s#83y<*WWI&rY7MvhOHS4eOB2ftmDyEMX0}zPh<2IfH?9leJNQbF zh4L#zVtTucz#X+k^`<5G^$kS$F2eBiT=Jo2Z?_krN{+?Lc|spQtD zah)a8w}^l{x3-}Hk+1(K5X{$Z3Boz9MqF+t{{-EflbHIe{a3;sA2lLoehCD909fFO zBU|$HKdAu@`Cj?M$*fuNo=$E6Wr+NSb9wR-#-*Quq2Y$-CLtmkPy%tSWbk_Gjn(e+ zcNSTtLGPP`7^GP8Bf3;nvD*)~vKdejyob#r(X|(&2*$_gMMj<5L}D1q>ZNu&Rl#Il z;Ne`&)m^c)lgPBF z6&X@C_;ENmrIP5khdTzXm<-}(*4^3r0)Q(_Q}2-k&W@0_G>vMT!=QU?Q3r?#S5e4v zp{ahmu=iGkgQV|C6G`FuBA(abGt`eq-Q|DM#;CMj%ggpK7I%KeEg=|CfiNuF2^aX0 zmb@1rKn|waIPvADq{Mn5|I;}rW*%A#1XWpkz|^Chm4#{TSPil-%qp!09`WnT)2aHx z*pJp)zg>2{OdOW@`uja?ZlgEBD2KKCeKbP^K)celSMr6af~7a4{CzCec~K4ha%?F> zlJuYVmH1r7P|+-#mb>329@OQZmh6J9CvczM7Xcr2Vpc=rKj0KUW~i&B>!!1Hd}a1E zA#$9LGj?I-anTj>6Z|-^C=kPncS&8>E+b z5Z6$)CeG8%f57zJX$|K!nTsmW@+guhhI~1@Jr0FF?BT}yj<0d8e7J;5zJ_x?-S$SD z_Ysps0bVN8c!Bba3u-&~)1_HBoqD>B)+nyr*vLDbT}FP$!3v3Qs#(!#@se~zm`q`^ zR*$+IZEGx7G{*QkAyxZu%UsEeutkmulFrP>Yel{b)>s?2dlD>5R~ONvA!M1J!5cF^ zkZUSp8n#B?0BTh4Ww}_h3x{*qnmg)!70MCfx9?Zti=lc$L?RxOC}S+q7$u)YzbD!$ z|J6TS;+<-To+`pIJBF(IpO;{GQHdDr1 zb!XC{GEP3T>XWTu5ms_>Arh6y?Ck6a4oQM- zhji}2?@sJx(_${Qn3x4{y7uPb5X%2ZN|D}Vl9sOcP@)Hu1Kh*%$a4;_ic;yiL2c|8`8eA>c7o{4GOgeeB7}#V=0hPv`&nIn% z2y>-JY#V*;e+Rw7-i%uK^BULG1!W^;YWj@WU+hBf7xSC1X{^b6XIE5h?VH5TtaG*~ zqi0oCr(a{8P5GklewgP?M!nrxz|C7W zw&m2eutOkoW09hyp4D8KENm6NVt!vP#6gaS=+yKVB3BuSH#=r86eivj(?16sHP`y2 zC;4j%9zEzY;bpZQ%BG{dN;$6lwF>%6v_PCSVtH!7pvrm$8p}Ek-5?1;316DVP&1mh978){j+931WG6T0|SYh#Tf47PBq)j!-f z*299Ky&OZSUGy%4?+P7P_a1~AUUw%fnAIYf4U01r;}1%ztD`z-RYLsN?Ru}Nu0@G?6e%dWW!|)%mE=Q3-ZR?eh7G6 zD7V5-y!{IMhkS!~^DIqIc1W&4@O*j(zjL~z6RD}TD?uF7!kO;(t?g1B@4yCX&SB9a z<>B?kR70ydo`q7D4I zAj!c+DV7=OV;NWx1zDqTYfO|cGRpW7=$+-Z!_m@t{4*MLB?}!(IQh@VjwcS_nKIQMlsKh4_0A0F4>;mqlqp6%mM|^QOH9<{v zssXGA!sF8g#uwWgQY@B!s_h~ibJK%<1$Z9>obc@;%Y`|M>paJn(Gl!|{ogGVeF`yg zhO2FM!l9vIR;k4e#ix(WLC+6o0o%p0(jd0!-)*a^m-nxH4a2*?%MG?F7f}e`TXh=V z3e_(2){WX=PS_WQGaz<1+B2LTbg9DcFqyX$it|Wk-GMCiQNuUxZ!oRBJ-()t{1biNRJ8^_Aup_+T{RyH-w!ABNKNtqri0p}d;-<8 z`(etsuOL3VJjzkRcrXMaQnUH40v){5wSSBKl-5eE&4f(vz!qth)d>kzW+R_-f9GD6 z8XT6OI_fa9l*j9IyJ305#r$6x1q2f~yNS&`;T6*x)2d*jc6#nCq#ONlWct&ZA3ya~ zRf@bxtCX$RUlMDWjH2eL>7L2{7bg{QO5*f=z7_p5B&<%l*!a@)e)o5{)H{Lp#D74~ zzmAM~3l&C9Jc5FKQ&ae5Ua9GsrP`70R1$4mTlRDHac}?A*JG?2B_(BbO${?2Uy?X7 zumuQ{O2k7S6X5vAzxw-l#%*rv==im?lx=^iuy0}_72#4ngRb_?&JsI0mDR^-{HH%T z#sSN5WJ^oS(25F9SVvljVmviMso(V6|FQUp!3{Z?^N^xpW2+wW;1iOQhxeE()h;j9 z(!f006E!sfwDgH{pjV`X)-eDX9Ua0Zz*^75gu0~6|2$(A@4sowF-?X%<78MU<>ga| zszm!11gV&?YG{eQ=LOmuA0H7obOl7W<)2Au@;^7r$vq0H;Tzezy2dp$5aH6i>@58s zjr`Tf)G#0Y6_S&aQ*Uo?WK5(F!Rddb8X4JM%Sb@W%*^~#H7zay^aDUo{m;Dl`w)02F=85i`0$GS*Z*RP-;1Z$AO8OGGl{mSs7Ox15sl+tH{Jkw(6O+70=q(@i1~zt zSzeD^hrj)Y*Y{5=U%^SwPoyc!|6+0e_VD&x%- z@f}I$Hd~d$lnlC0|Nh;al9I3UNAE1o zNjf1VC9KraK{2{wqr!Oc|9%L&DjzaFC1uQ)FE5L1ZU1k1q}o4Yl#s|?Uw<{VwDA8C zbSQcPqF?B0M?^(Mg392nk~l!i&Qp^M!R_v1Oh!aBvt4d;BvibS!Oc6%ZYM{_=&z zQ19#3~fHZvV63N{7bAI1Kv=_k7jlw#TO>Kgew#Uv5e+ zGGbf(OD%Z5od5rSc7X34R?4~T&iN;lQ@_BHv*l@8X&Hmi<^m4iGU2Cpla_S!neyvJ z?@eA-N5b^~^Zl%NZJp-yKvVHWuHAlfoDRF$*Lzo{(&g{$b-~w%%9LzNvnk#k?!#so z3la3m4aY+Xs^uArVPYu~tebjCDwnN%z-hB#5foU_Z;Cszg+{J{m9JV=;0YaZocK!U zRH$n(k*eo+nMvDX8H+=Of%-=i*#9BtE^Gm(<52I1svOQll=;%pul&oaPh9 zr$ANY@7wgkiuHaHP5mi44{QpNyd5|{%TE|pEz$hRdR%;;aj=vs4CbhK{tC}GOR}>u zFR++_Mu!#<(S}NM4O6eKB$WLG9gmBO3M=XU1?j7≧61YL-8x>)8+1h$UD z6b`u7H_4lmk?CR<$FH8epBwI})v#)}tp)-2R8Z{AU6Bu_V3u539G$Us5F_xd?KL`$ zj6t82AvK*nY#f)Bk!*{V8ZTo-MO=2Za;o86P6F71VUc&b19MKr^M%@fi`OOGljnK5E)j`Jm|F|xE zYQI9kDrzM3=Ib^)opFthfI|p@7~5xg(sdpxxN(#;6HR8pZ;J>eBFNP|G>c92&rr!i zkUmx#jIU`z1pu0v7brY+BN5uKrO&&h-19)*@u`3E5e8m~p6h5~@XJuS_2@2mlFGk% zx>YbPe7nNQacAPh<}j&+2~~=$B}v0nY|4(>Yzf?003XtK;XCgtj}v{hP>I#y2(-C7 ztMGDMW}9ie%s=$+vgzsZ*MT-CWk_Nw0NSix`pq4W+}@Pkl)GwVKWW1xfXzM+bo-vd z7QcT(OY31j?(dI{an^?3=Dk$i-?PAdL&0q)<;s^fpc?e!pgFr{7O^Z?5r*Z7*ms$H zy3o@U>4UCseC9ZbyBo|<%#RSnY{s8j?vy%+S5)zgM#FLigwFl9e2v%cI5Bb8`U++& zJIUTl?6$Nv0riE(jbyWKAJ}v{IKSk6GlO--1wv}I+HCxPiAp*NG~=7?D5#DIIBT@w zGrtigA9GNRFq}^BTHya;Mn0x84qlSiC-cr-TI%YP{)IL!Rv*lg+xE94=@5GozE=k4 z5_tW6HE*Can_PFEk?1v0q7R9*_u34Ao2l_cW%Y+iLD0S6upCqFg=@ayq@@ph4fk_x zdmgI1BnlreNcupBRUR4s!Ru+WJNd`wTx>%xx+9*IQT?5=FpJMp^W7|8*!dl9~F%gv4Ho4@tA z9B7reDN7t)+Ky%<_&8!)_gyPA6S9K7KbiPg7_x%kH1z#HdAuohoI-gDf@oKJ0 zG=x^>=@q`;$Zyuak5Og^zak`qrz_&#Bp8&Rm`LMdv^Ev4{Vr;5oB1lJCg%m*M&Wqm z42?)j)2o?8#e9R5L5*54{3{vpEP2Y5t-wNQG zho=Zve)&*+bGGFpT?L|=650RS0Vjxk4q7=`b zM=&u-KV{=L${ekE@J!S#mtj2m^9p_?EYyT22AGx+&MXFN{Pu+F&h<{h_um!g=b!V=$0i+Z%g=pe4uLjJ{dZ4 zJop>L&O%CM{|Cap=(`iyGywuOov)ZWISM?*d~Jc&crMt_-uA2iN>3*&I!;gg!)^Yx z#RdIVgF=4GQI{=m=JPZv?k3>#&?}laDqQ+9f172#0ccVZmp}G_zE8Z9BFm}rR>+tU z9P{agVM4{a;5v=6 zT*)D3?`(XdARl!Nzju-`$%;mPUX{_Mun)JutO&Cg5}mh_E8=Q{EY+{5{DB1ut63c6 zOR<2f?HJFoGKjKiIUXJ8X1^I*x5G)BXSyy>D4p=fak`Aoh()MwkxERq*q1@33bc-f zD5)HU+CxiXa^S%r%K*ayPaUf4u7q%73Eii_Hm=PN3t4X)*CS#h+6mHLFg9d+tJEzrN~2+2`n6E@x8G+Cq+?{ILXVgQ$WNwZPcjJ zn^!sF%NKV!x}J0HOt0Bfei5lw-_PVdfuM?CuG?K?vtuEK5<$dLZ1?ywTCnrM66x#O zIrGLG-hz?h;}Z`|i}SetE+xE=Pe86O`quM%ngLp@>9C>;8|cK{8`e9!t9ze&e~Qvck^Y`U}T^tQTVJ=sd{OO@e=2@z&ZZS_$l$~GqqMJZIm|B_n9u@SLcLeC@d0cafc6Uhm0Rp*KD6S*{-HBIgr|^uW_nS9b9|_C$hO5fd zcw2H~wRA_5M>)Q|PN}W-ouN}VK73Pp7$M@l7Wzw9q9*>lWi&*{jI-zC2#MO0tGITA zi{0RLxq!WN5;B59tS$9nxM(Xws9MA=EOM>Gz;Bi0NvZl7&*x3}q1WsKfICAKcYV}? zXzSh2&tJ%Rc~zYQCXj^6msKKN-r0g_-bG!Y^iu3@n^qv-S9=L_j_7D?ir6T5`RXKg z8P*=)*6NZDUdx!yKQF9!xc0gKc$Ej8K#iF75ECIOVAd#9Mkb)oB4*s%!n@ztQe5k) ziEpVhMDoOsoxwy3Q??l7Hhfi<_6g>+ zm{R@hR}$Ac0Wr5(AIwEjkp|%GAQgZv+s@Z78ag=izPGQZ_sj-z-rUnT29wdUVbYW`FbUd?{c`B z@(ZU_;c@$pv-rw46y<>1X75XU+!!1khkG_trt%UkTJe`~(}5nOkm4@8nM8jbSjVRy zbpFpv6S}9-T?F z)MIjEbU=&SK%{@jc2_X-rIZ=Bk9ZrGoK{VrTG8bQ=I5kycTSY21UJO=pqcddQdLqLeU%FU`J40|L?nh+%!`rZY9^QujWsX~b_}L@_ zG(23&&`}vhu}vq-Y|`8^I;Susa=lvwURm;6ak=ySWcV=E+Rdx~im9ldRqKg8HI_Y0 z)a@7!9h-o4D-xYs0X|9K#!p%9R%N$X_FoReSMgitTHMf|Xc=BgP)z?cHKl(|!U+6P zkZz6foP%x7l4GEi=!bEQ+MAb9nl*5*cGnikhJ5$!H;MSZbL~9EaLRX^545Mv1YL@2 z=I7yIa*K1;y~}jplmKD(99RF%1)wd-$2QID58O=&X8G$Y{7s9P_OaBbI^f!=rmh1` zcQ8=J4~CBXmi7mnS^kygH(fb-Je!Kc%K4>56QKSI$$P?|C6SRBCEr=U3n1@D(kpSJ zv;BliC1i~$KcI5S;9?RcQx{$V|HX;37$#K{k*R=QRE&rQoeAg>~kYZnFK#s zF4TiRj`m1xYhKZ}0mfrfbONQqtrTG|k)K1%M;}FzK8>zDpY2@hku2UyVPWoFy9RU! z;Zl%v#hb;7_P%u3FCpyX^-;3y<9)7x6ADv2Ow84Kc~RFsXga43n%4!acSHbsCp7h~ zHa`zH+st-E;^HUV9?KI(chACMh#y6KA{rBG*|T-4kBHBS^KtA>#sTZ64w|nJ$b5h` zHgQD|6@VGu?T;%2{v&vD=G1NuA=t+LPEIZkb8EN~Gh-xowI=1s?_8l+U7f9*^_Afv zK1hkUQt&8$A$~TcnZQJ=N|z6-?!gv5{ftc!)fu!#e;gq8VwiNSxlTIwdXH1B!<}Hz zI8cyTqk{;J1Gtf_Wzt)_+qcAK_wJobh?K}{ibv{DkNpsfHuU0N1R!R5+jEkknZRfT z9rSLlNb?k$#k&+N(Th&Fd1-ulQ+W#>NWc2+M=Dfu)*FTyx4zRgZ?+&IB2Z#+H2;%J zq~40`QS3=$J4J^=+6n$9XIRnK6C(EX*kry~f_;76=g)Jw?h`}iFIe<0FY3n%HckKS z#W(Yf-r&tHaM6$NtvG%SC`m|y0`D1C;=o(urM27!0g_11I4hU%HS^*xIH=wxQ-GiF zxfKB-j2it&Xn2~K-yFoNVM#cGi zKjgjTTO8f;H9R=MA-E1iAQ0SwyM*8bcPF^J1sxm`Bm{S7aM$4OI=BaSA9yCY&pE&Q zdjEyDKfs0bba(CARjXF5-dhty;!P{rqvJs&HjpQFdzFd??bV4$j}u_oE;nbrOPp}O z-sw9#NGxl3?6(fxxj_pPv(Tk%)4P3_ALWa-3~m@Fk7lYA8K)eQP-~+^Xy$4odGvvL zTMNF7^3o>;FarYSFU2a5{R>N+EObY{04^J!-n4l7`MD>TGl>NFKn}@<2&}FW;o-H~ zPz{Dc&BNTto5WE+Bqc@QG0KKROpTVk^no)sr2KapLqq~F8Jaccb>%7aI)*sNsz8=& z;O$^{Mh3Gbsz0i-SRj!5?vCTwK+U|%=`t)Ogdhdk;AW?6zSfUhZFm)=pf1bNn?|uJb(l6V!h=8#QV z6eZw`-UCOyYyAwX%vtzY1TA!93zv}QD6og)o4Tmgz?1r;?zUEjvqN=#gI%W#gY~ZK zc73<2htD*YLY~eMK~K_eC)vd#_h}6t124`7qUcjKo|(OC`Wl_SlrYh}k%~F(3uI# zUlKZ>nEOY8XQa^s-;H8jPPMI@=z8`B=4u<#63&;9Th7lJZcqDqjr8$OP!W+)$BQIB zxCUwgj(xYSye}u+=~QOd(h|(I_co(g=OTl^f=JrjoU<|$Gn8FFw^v~=Eg0S2Uki_! z2*u}7N?6t!ioKERAxV{;py97{`~H(K`ZW5JIJX~Ma8k**&A0&u1sAbcXirxkGuHAi z?{W4b^coi;hY5{gJ$*OQhzu`0yK-K)9*uz4o#$E<_r6}QSQ}AlTng;>Wqk~<`BOh6 zV}d)xFC*)pY(K@mHhi2X{%mg3cbyaUpv0i;YH&xix~Y+9Q8yZ~eyef0_g2DVUqXlL z_)IIMl7w0CiM-8reswYNq~2rVg#ga`_>Jy~-i*L%T6vLoy;*K2Ka(OO zrc2uV>CNigSRWt5sL}Xx%;s`Td*6wSH-OE_cJ#Yh7XoYQw4;f+zZdX z%lBSFdIYcbWhLb)V}L%$ZBTEbcmR!ORCjj?Skf`Fx@*XiT&op}*6NY!BTdd3 zdsU&aO+j=J2~b>o1ayyoxR`+U+|cYWR+=(CwI%!8#S?*2yBZ~QR3NNqBpMwJGeFi4 zmrO_-ZB?_{{=$E@lm^h5^|i1>FkBVAy(=+|m7>`bQs4%X3Ds)!ND(i_!-30N5(HNk+kt7jVJmm{J_?PdjrMu_Wo~tA zBtk!EPJ`P#ocH}?TD~2_`5Hdf(~J7b zz2IF%eq&;pnDV~*cDl2*C0>nUAY zk)F3qmshy4UtAFFbRJHPi7PMkZ`Xc5IGiPx;L?uueJ#^gqN4(tQ>FrV(B6|%g%(x6 zHQv5RIURJ;YcQ>$!g{VCm2wO<+&>T$7993%x2sRpm;~C-Q~cmO^zy#{jqd)@g$(aa zLFG;aI*8&PqnwK7Yke_y6AJpR%M^bL!3VhWVf~8Z3gi$ffZgp(-bM*QN}Q5v!0%T9 z4werYg$dnRQUTcT4`jW1Pgg>$1=oU+M|mCVRI{(x&p&#%JJxlr!eqhO*L0h=lHQje zQN`0pcd<(Q+LU?9o2~e1g7Uoz_{0@A`*#$wcvfyVNQnio*zM{bd8z1DKq=qaesK@! zcSLadi4cvjdN)_%3u9NS{O%F^@$mS({tIU1`6kkW1VeZ4dyCuKd*W3HWA}G(f{_t3 zlCNGha`D>7A0AHX{i2Tczmu*sY721ko*O797L7lAE0gNp6BqCIuQ9ve@N4&#`>w`t zq7wY63>uE7#D~XoEW+>6<=XjXXYo(Uhjp~@&vC* z>Qc1LLp83@vBaaXXU#>re`C>`b?L+7h?5&|Uo*OQ+{}>G)94lk4-+*J5u`CtTtq`6f?pncBlc62al!8P4i82 zXrey0|Av~k>V|#SWl81vm7M!U6w?m@R^oQQXCAg%;d-@5ne@QFvx|<&x^EPN;#*&g z9o{3rUt}&)JXR2)xh}1JqanzOs{05wbiS*ehkPmvgR1VE_YY4YaSo>QPb1hLth^bq ziTPjT5=KSAjf~=p81I^vH4)hva_JV_4SoN-?%~ zJ!>vQ_!~bGRm<6n^V$)}%hTz;jLihikNtO7+VA!65GF4X&W0bEPbV?DQW}|FK8Go+ zCP!6N9cfe(;I$yoiCVK(^>~t=sr8I6Y@ujP|jMoDQJ}7?++T9h1!4-k` zZd&=;GnD>Ic$NLuy;J#6I;wN^sTE&A?uE!FeJAzO89qzWmFHVd9WpNxt3;-rdwrLN zvpBcPwJ(r%Zb65s3hNpEVe&VfSy3gGj!;}_&*iYYlEigFvi+&c)O+Ul&tn|YOg6oI z=xrUQ8CIt|v`(x=n$$Ntt0(+rHg27$lZgChokou~+055vK2%g?0ui|f89nED*g!Rw z!>jvqvg(N;S|lr>dvd?&$j&JBJ8p8JHSk5{USnlH}Vsw!5Pzp0XRb+iS>IRN@fh@nMyn_z_5xEl+#YPx@Y~2A}Hv9lYOG&)T&V;hK<> zb&GLlF)8`2EeRo`*5awv9k*Y$n$ew$UMxmOb;OR5r4Uk`@7+7!M|MOpDGjC{Z>7Jn z(y^jjCB5%iIb<33eiGp4=OGTcagAWH8%5osdc;6awD!-2IJZ13q!3dv2=DU|+A(HC z6bV$F4lJpkEqZz3avduTLvKbH&APR`+K(4+io~bex(%;eRR0W9|AlX7IJWxQLBYXv`JyG31hjp;LWE&3=g zIYLxpe}9Tg0#|uD3%5(1o@Jxp>NX%r#ihJ>I=;#QKp7>Ydx90+9WtyNaEu7{UjyH_I4xXWi2?D(|U%wzwgec}H z*3cIeWM|{)ZWyK2Qqs^=2`Ai8JnoL*cH)1kc0v>ss=~?f_TWaSrQ(-*iR-~M z<-}pEDK35_iiuUG{T1By47b};>F#Z*?#BGLgIscYa zOkpMPIh;37hy71K2{C}0#*#?p^kLTS%t}vz|Gr(le7^Ya>HX}l4X)7@fNlfousPrf zY7<~NK7L+LAS(ajpiTdW0#V-})c8k2Kna4Bz#5ZG={aE_{-obLKPAThC-MM+3cCye zL4QBW!^1VgPH7Jo84iP=x)S^*bPZ01yrl7e|8pgKj`h z5Q||2a{XocXg6v`Bf2=p7s&pu3?bfsX5l4iz(oh_Ix>LJArA8m-euEE|^$ zXxByX^8fS-N}q@UaEgWFMkC31sq!V3=M6wq9EnQmwTW0%BMcB~z<4ZyBX1cVbH1w4 zc`O}2%ZDmV^?$31MqKUz{p!1el@GEcm!7x*;_yoNm|rmgEKjY1YjB&ZauDhc97NVd zpKRjx0(GPNHPoe8;>rB6;D31>2Cogk#4Z}1Od%<|SjHHnPi*DIf$iS$dX0K<91E_n zSlM{wEB~PaxO+M96)SKOe3Su$)&D(|XlH-J1K(|1mI~>I4SVNNXHw&0d?bI1G@63U z5oD&Vx3H3)P|)$nj}&r))dLNDg}XoXyKU)T!(}`1rM#b^mS#2;+(Gq)Q?96qdum@O%qQ7TVnjLn$RxY+xPz(1Ju3OaIqsKXooUIAzEdD zz&PS50e}cTW+(P$yMz*lSs*B+iSnveATpT(C#ctzk6p}JOQ4RD`Bz*d{gU7Z_?Jz{ zb?fT?w*n-n(CPO$T&z&|%zV6Z;J5Imd`&7zn#cu;D@|a2)RhRfl693p;(YF>DSs&+ zgC1os%|h<@cbqO1J{@;D=T7vOEW*nEQZ@EX#laX*h)#a1Rl^*zm4bVhTYp`yn#!! zwOv~9FCUox?a~0~pZik#A+;RQ0M(nr#Fx05TeiLgPY#~u>e=?&onc+O!%%5H&#S1f z-|vU9H@ooQ6jk447MBEyzYP?p|MFpLV_T*rUN_hZS<>kGRO7b)^}m+t%%1(t0v9WA z=W@%Vf@E5Tg;&)5V}|cjj`cI%DX)5)nt_UC;L4O=po)I) z2z}S8*Ud~DVgI_8+$}o1>uSJoM2z8IrZEY1r2_EiXLa^E6Pyom%29Hg7;=nEOzA!2yL+qBIG@Gzof*rfTdmdC2;Q zt&vCO;VirRw$jm;PJ34W67)?k)ItE#!IIteUu058Rk=NuYQ{aFt8$;5i1+3GZG=#>HW(}7 zZzkWeyJ@ioZ#eFqbgu2$ZFsr4Sb!*CN?f=Chg-d&QkP7 zIfDcqw{Vz2R6?FwJdF}B`7-={e2CxT4>%*4oQchX9FGt7mH0P01fgdys9j_JPg5v9 zy-*VY=-(z~1*AHV_!4NiF&v1Eu}s;F|v~ zQ(1d-i6uqV(7Jxoy3!0}z!qYd@*enX&5Jf>E==tj9PPi?YAojcVtg{r^77VMa^HT^RA!L{O% ziM>RKaIKpyXNrDy&T%Cvrmh?m;DNJV-w553z0G;ao1>~`D>uF{s0q9&UbwlufUkX^I_$X#R3rDchB4kV6{n`Lgk}>zP*ja6apa@HU z5Gf`+BBOJyR-p-Azlm~zdE&oxnYC`|BmmX{z!y~OX(D>Ny-8x6FP;`tz&?Un(Vr+H zKWETzB@D#f?1uM$W80yV1$}%2f1EOnJ|*eW^(|v`eeF!A>#R2t`#S-rhhFrgRjq$9 zA)Xw?G}}89wlD|lUDJerUCRak&$TA_5ptg#{l0gEe7u%|Zbq8~U4qVk*@Ox1;Tk+7 zy_sG`w_D;w-0i*Uct{zg0axUIbd?d~l7^;YDXOa>$hl!8^T<`A2T%a0`1&uCtBC_6Rh?9xC9k(U|_j};;cqp1}(=_O_wTK2Rx2DRA zs49n?zbwe=Z@_S|72KDRp)USBUi2B~HJ-G$H}%y`pt>`@_l7a-Tl3{ayR|Od>_oqAW!(Hl#6eR82WKR5@co zG`Vaa#iDS#K8!s7vl1@M?<@sc)gm^;+OsSUm_AcLDYDff3X|Pv?!rQsTRB3D!oy*G z+s5&tc&;lV6UP{Ae~V(P*mfsonYyEZ^9>Jx@QDX(IiW-8f>D)<8=NYkr{N;wy5Mpr zO3`7?%~aleHZb5$+bzrbhO*Gf>5xj8CfKk236$!{m<>(=c=)%g%+0AW`Lv+*2wB%O zws#Ljz{N7moP0FJu+IDvB4y0Yo=lB7Vit`6{Qi-Jb%Ln=%`LBKPXJ(vp9AYVin#H1 z8^)nvBD$2N|MWY{EV?3py|NG;CIpjjA+mT`hU8gaaQ+t#3JK`Pbj6=$)99W}nVmUj zj`gBdbiLh*Mmn(--tIfo33}rPLtL}=`*A^9QA4}avvirGPHRT(DPOGc<;gz9eGeTb z(9iZhS(#mqET0o`*ulp)m#2QnApkXPayhq~Bbz+=%4?UDsHQ83ob`?z>xGQ)vjJ$r zX)f>YJA&6Mdl#uJ`q5Zx>J`blO}t7%hH<{1A&GQ!xZ5qw3{iDD?xJ9OZ!7aV=rYSv z%_2p(>4|#E6BvjggX-i1_aBuD=}V{+K1OL$WjmZ@8-ha2f8A64*IB1|i?fZkY#>pS zM|>2`Ijt0Q^Q}}f?Gz3-Y+8BeqV#BWN8)$Kt>bE{OUG93zyH)zOM% z{Xf%lns{XjZg!nwVEEb4(HWipvk(Jpl=``vv6s*+qL=9w+R*n)hmQ5+3vx5DBP+)$r&dg<`0 zTbY(av5JSC-I%+W{)C7kIGP{Gyokd1NPs9#p`Y={U*55I-xjoI;H!LxXL(nC6fqom zLK0Hq8DzA4=5764jpO-!E`C27E#8bkxL{V4J_(SGAGJVf(GOsq#$z?}uRuoqFD_mX znU0`?ZX%-9ABk^wij9}39_uNmNb6R?FNQn9ZorJUmNeK4JIfuy^!eE%bq=~Du9DO7 zyFmXd+m#VSmM*+s`wD4)kYRkcv2W2#p6{LGdF)M(7VJ!0*sa4wYLe%3jv*bWI0lT(<~(yvAl{eNX?zqp(k*LnT1_`>M&czj}m7mrQx;n)-%q_S&y zs6rlJ>3;VMzU>`J0t%K3NbAi}a^ckal9gS(?rE3!K(Aas2vsWP;n?z`vFu~oGn{zT zBJ8Y`f8y39CJ_|wyCqStCvnIcGATUGNJKt*fd98ka@k{wAkB}JHjUm9zo1+0XN!o# zGK_&q?J?AVNm0fa*o{WTaDw4}(Xss3#+;L%)u^9$TVWnB#7N{I@U=}ou*Qwh!x=f4 zhL7k@+co=g7k2;njb1zy(RR-9Vtsi`+F$zH9KJ6ll;F(MzQD~1X|%x{*|xqT;iOe) z@FKy2fdf?WN@@TQH7bFbw3V*bHKYg9KO_}W(ft9j zFPQq-nybc=6Jvo~SgaMhvj!Pe1L1#`)XJT+RPx^A#Eu3lc`e>Wbw$Bjr ze!sk^Sma6G*nWS*1C~p=&ZAF4PUFN51QYT(_ zG*qvIeHP(tiH;c{?c+B2^^cL~0uPxbMXooF{Gu+srugOBCWI4->SRD$8ZY?>8awot{( zdY3=#pBkC3*glJ7gKFG_Dju{-_}S2Kx#dOn`m;_?RtWS&;Kr?L6_Zpmc{wrjP+~GS zl3K^-7TSE`^J@@)GtAN46}3Yi-lsCTtcM=mFKxh#(&ggQg{`%Ttlk>{44BW|2{y1U z>whQeV2R~?HDQzGV=9H%8`LGlhB)Jqra^^91?P8y}ppRY!hkaoLo@% zr{nRZ_GMqNly$~t(h^yhpzh%$kB&q1mocW6XKEQPTgMwV{s=l6mt&fkL4Tg>wM|7+ zvp?=0(e8n^VVAyfYzl-%*u~a6Cd~Lw`UAV=)2wf9@0xEQS3Y3u#FLbZ`-Y);x_{T$qLtQR16sl~`P=Np=hNp|$` z6y{mTHkmmnEjDo@1hlv$zWm`N1po4-PMw-I-QNhh$d#fn{X4eSmmO`^&)n_n7aJFX zVn4os{ncpTLze;$j}Ix{ar4j)P}{uIDbnNKlHu#1Je^fnjH2F$U6|Kqm8{l66`YMjADuuNc^m`SwpLjTW zj>nj+8DSO2w7EZs#Txn=?FwlWlY~0_n+%;yN#>lT+`cjB?E)SPzLH%ijB;rSeEbUZ zn^sj5`%L%WNX0LzYRSvgdI1+t#N6QZj#5r_Yk>rq*QfAmdz(8`=dQ-l&4_8YdgbQ3M@i^<8)p7V))oWAp*VRf0CM z7IeYX)8=9vuce(+4^hEu!u{DfnKx%PS;(Lc(tireX*0H2?+6t?zicPHAMw)E(l*b; zy6PsYuF^I|_sYmjZI?|$O!$c>D^U^XiA^TfPM_bH_CHxjqzorqOpQrrJVZ|gQeBCs z5&pItlR8_^VIo;oMQk+Yl$CYacakN;-C9AVc|UbLLg=1H>QGh#NVSU+ZdskLwQq}mVbr5<%B zsO5+-qZ@T#I@Xh{}bXq+Q3nTYLw1+h^NYCyT9z6g+W%rIgq9jXohNS&e-Hadf zww(;+OQpWo8yEqCj3k^hBRpM5Ee4&-+T@q|TL_=)ct#QKu6Wij6aBJ|iapz;j$3$Q zZ5kD;@0ct)uwSpO&r~nb;Z+z=5Xv&RHRzU2Uz_yU&ugp4;LSU@Szs3Mgil5%Z1koC zdFpstF~O_*?T;@HALIBgzYdMrhONzxn2Z6}74qu5DGZd(LZ~dH4+!yBk;*Jm0$v1S zmtB8HMif<7Sw?1XjN||UIyZZ%_nX2{sd#F#DqKf@UfJp;du^B19_>#1jT<1od9x_s zyB+*FVEJy0x8V|UddrL0H87r8;P{RF3 zodXIaIFX!k(SFreYh*+SVf94Qd8UQD9>@e<*CN`wrY;~D%og9eEz!}NZnl3m{7lKa zH^~DKE>YZ=d}d^s-LH||-;L=|dWFko&C){ii~8awFuxd$vxtOl)o(n%b;kt_X)@6# z{OQ?u!l>Q96CsNcTyf{|K@>crwWKFYisR43t$T7hf6U+Ktbz9q9tB(S)@2TO|I;BB z1-ZdXg30O2?}G8fB+a9_?XO;OwbP!Nxr`ru9&tF#IB38bL7G=nCEUok8# zGN)XIccuSUS?TLbDJP3AhRT!_f&NcWoDZ4I@Lo3o>e4J*U*TX6Asj1nlrCg=P0?}4 zcPKbY8J^1j$n!v5edOEvK;vwn-`qPL|8o`o6D2`Z?i{23bY1AMJ9jvAv{IDk`eYa*DZ?kIXKyD+!e)x9f539T1@}LAU^6jmSi6u_o$Fr<-}OUJBf_{KeOmtA zB#ypIhO7WQhYPvbdv`vLy5=!fif~{@`zjzv-wo5~tv)6=*1!$izdb0QD)PI|M(yG5 zhRT1QjMhsWAil93)uEJw;{oLak@F31i)>uK_o}s7VAf-G0FiPpX7S+fmZDJa2S<_a zjFU6IAQkwP4?qQcP8Xbe3l|%I7m)>4(LS2nY@}86Ec8Jv6JP@<=gf*)wG=X=%30~4 zVkT1vn-{_S9@+75tXR}hC$>51A51`@xHCs)4f!TXBXK%k2})tbT<=0@_SPLuY`xZw zfGBn^c&QiBX|<3o@Rg(g3c;wblcl8-yiJ)oqTf=QAU;2%=?_F0xsJPg$XYEKTMvlA zh9RY@4|)U9x=fhE+u81=p6avV zF$9SV^TfA)Mhe4Tve^w!lg5vGuDyfr4)`2W?l#|+(ONX>813**#9!Z$DoCFXcVbS# znp-=cCr~d(V)0QztsWQ8gza_fWu!JU0U8?*d}xetx64ah0Yi-3Ho;?USw5w-^ss&7 zx*5^%WRH|Cmv00_D%Sh7S7&01U`ni!iEnFPp-KkEt3RuW;O17)1C|*1l$?Hfa>?%1 z&e0f6E5h1Zucj5Er?$6IG0sFxi}3O_$#<%FyLYZwc$;I>kvKAvepdJn*HjCbWC$F`q8jf z0i2XxP5j7`@Hj1`=yUmDx^cnm0XFkQRbcw;BR83t49SJ-BYCR&H;vkET1zyT0Hfn{ zQgfU@%`ccD|&a2<-i0HvT3u4@cLbCk6lU*HJ& z51{kT$ysW}ua|y)WQf?!@SLbNqXA|yaLjPpR1`B@x$m$BVUZ!ih$QVDJ8+e{b{kJt z`t&N5uIjeaS9e;~A5rlhVER%)7|aYT!WDh@8# zZE$8Bo#jeUA^)tK7t;MH#6K2*+M6&tcV+RDn1*8rLnO2eZnA3hl>Bcas*o|v6SwF> zs6y)ixF1bIT<7S}+{TjJdW-@hy?_XakNmK8TUS!8+>Us+V%wxqRO6Ub0)zfRG2})U z?s&-W0`$>JG~Cc;H70^0}??Wd9#j4tc!|BA^h zsnmm%9OU?gx>-CdYK@5@0OeL&3ioSHU%c=io+}9^1@6Uj3d;r~v6)8yrPWdy%%{`} zCzTnm!`iIge#^L*YG#@*E>X>Lzlbq|yb){zWeCn3JIqL^!2?;38x1to4Lqu&GI6g@ zY$!Q>Yi9V2tekAxJ4{M06Ra{vVH;(|asW}Q)^G|&@>Gh-2-l-pw%>+6F{9$W5T(_z zG4*}3PfveD2_YF&$Ot`u)Px0BpNCfsc2&!pu8B?G#_lu%Gd)G&I90;Gq8Ply+_gM-( z!v*399U&)`b|&z1046V^zQY=|SN=)u%mJIfVBp;LaBko~MI$ax3XCbU8{sB=2wHYW z_(u7xO;*cwiSl;`XNN32cb22_FN6!F<-yD9G^woNItffQoz$g4ueX;$4D`$FhvqaCNIDO6pqcmv>Wwckn;6 zMN6PY$8G3vSWB7*T=(A*TYzW2-jBDpgp66$IOG6ylrDpQ%fJ(J3FT} z`aTYc*Y4zDXw^Gr@e|`5KCX%5>-NV;m56D0|GGuT>)`1*uquS(W2v`3pdlC3!l`LB ztBuhjKGraEeE&gs_s{d@GHIacmR&^_md89Vzac6esJ(O{#fMf5awBjJAt$n)CzKe$$TB4LxePDGe{yy;Y_Xaz{d`wZ!r&k{MKNJ0Pk)g8iy{)yOYj z&YZU|_JZJ;FKAF7Tl;8N$bx0V?&l;j1C9C6@;Q4AL7mOjRzIQl$nS(igxmbrD|%(= zC*iC|?n0Svr*DH3L7oE2yNvMS2P8kkm<2Jzmc+VHxlYFu^feK>Ihw_Af*&&j{dAU=3_xPsmlU7J?nBj*17ua;E+6oAKv>pEgU z5w9;^N2_a;rtf*_wrF>N|2Uk_A?Z|Vya$!Aet~V}Nb@K4q~$Eq_;^reu6R-;CSZn0 zYt*7>_j~--WhtRm)f^O{RKg0O<(kdhD!ao!5aWEmea4_c(72Oh!Hzxo-%knytqT6-+(^lC-OOT4uwvK6aU;}g^ z1*VQblo|}b_n_|s6k);#uU96I8qR+9buh}RUVT(+C^*gd#Bw`*!*qvTxFr&4Tp|rC zuTMVva4ttNH(w;}JGxR)kbGIdGz8^arm%Zo@2_?RHc=@ov0G4QS=%k1!_f>Kq;|-< zd1Hy>bAdSx3v1^N>jjr}-TbkTJ@T<-oE0 zwrJbVxzxf+0l)^HD`LINexe3 zzIGLH7bQQJ*N3nEkJigN2RtS0F#0&l0ZZ+-l2X~1^B;3Z^g&Gl-YI>AoR;w?3xVgY z4>$rBSeK7|P}^n=T^KrpF&D7;Au5W?XZj81L6-RACBt?_)K?c0_vThgHHNQhW*CX| zXNuxs&AD-Wg6y5G7cjCFTT_}Y`=kML*=z;2Y;e7pRRlRZ9?1mjI6h&`k_dhruN!WN z?;|B*(e%E1@B?5#JaSt@;4HvK0?zwVmPp1m|t{ zv-!S%Oo$gj5=kjJ{X3OSe&73IAwIuJfEDF@O}JqAw>+g5B6N@xnEi2rQ3;>S(i@aL zpX&xS#%sh_mb~u!_~k=%;s<7df84Dwr&xZ-zmWC$Zh-sUi)5?zuDe~mx`SUPFS99w z4`w&F*ukPI#;u?-29vV7R4A^?L5-U32UP+BZEI#o<4cbWzb{<3hS7D|Vsuoi?q^Hy8=I^}x1fJNK zY?1J{$QuO%GtK$Hs^iHClzcBt2PIPOLsp%0Ue>o9=v?Jo)E!A)of)e=AYo7jG!%V@ z`C|EQ797rlr9a0Qfc2<^z(m8R3OQnoKDMMrZ~7lyS7CLrJg5Fk1c0TAa%{DCupAP5 znNq4?*wnK(0x#{cxedChtsp%%B?pA5zZJ*tVWjM*n5v2I1B3ZEKGFK4kiNtBguFkqPBV@_7q3$*KF9#2dL8*Waq z53F0KjM~HI-q7u%6%6^qLQk_H+OH5S>p+4yhON>{#s`D|#4~m~mm?!yIyqAUCg!_& zOCgctU>HuZ4J4?Ia>d1~twjcXS=~{-nE8=Z@aqRQhXVcmil>XueHJn zP%?j(s#8NmaV|#k33KIqAK1=MJep!bk^vs5#Vky|YjA{}>6B{R-)PBJt#o?DTLLu^ z>6G@&hR)INl@?F2l_3WnAjOcOw9i`*feVdgG%s15IcKXCjGsN-Qp_lueukR)bC}v; z#`bbe^*P1=DH^gz~`>DbwQ(1)YcVS?9}V!tq9E0DX7>|RscB<7&O~HvyW`ZcGu%Nx47#1v zEsix?xxHTEtHxBDcID?l*n-6aVPktSdZE)B;cuez%4rcW3}p?{T>VA+qX5IlrEJNa5y6V{O`#pJA{Sl zgUhqwf1W!E+i7R1_@t4(d-g#Q6VkL$nQ&B5)~CYo++A&Gv3+%}Ru)oTcXfpq&=)B& ziLUUAmZo&Zm?WGsWp$@yNbIJIo40D_qCsqbQbEb-Tp_}T#x5;-ZKZY5LISzbs%*Ni z@rY2`BwAl)e{YxAvAmS3PS|7A^OPQTSKZ6@He{tBovgYFVuFWj$n+d@5w|GHmD^c3 zFJ+om)Sv#@iW1KFK~I6UOZtsC<-K*PI)XS3qF4uvvO)EyLBE`z;*Xrm&)riUoB5%#ZL~uKSb!m;et0Mgaw39iq=Hv)ezlUu1o7ncUET z@rd{o`~iC}j48D08J+1l!?~XFbb0d=nF*#+U}I0QVAlSuXVh&(F(?U z(@rI|GDr5OPTAKOoQ~*Uu6lV^zhv=jgaoIXzwE!5suOPwbF<1`zBV$v9safWwNSmB zp1el%5ki6MPy11}N6y%{e|V0|W#kB-md)m@Ipf5L89wcv1$4rC@$c>)IR`dnB{5TR zW2{>`#oySrn4X9KM2jfBBaWAm7*%ceSlRMcDyu)#tUKqD^BAA;e*S_lf&Q{>!OaTf z)AKLV)j0!??OAwdYNq71VaDXFN2y#~3$Z7-yQow9N|(596#t_>T_mckgC#UW?iGLP zPrLdKhE+Tn{q`81Cyci@^?EEk&&{*C-t}+Ir?~bkhy8XgjwcD?j3qyqmTx6xi2jNg zdv(}!0F1G0QfTZRB75=dP1q&qi_JQx=Ob~70Dy@{K*=1GU4ojRAfk*=V6E?oQKuUj zg_hOIXzK^hU1%AWFTkId7M+&rM<6-d8Zv)KM}U9+kQ_)ki7g>Di0=^ah{wN}vyWA!@*#_KjO-p^$L zqmnABfs;0TFntqxgs?SV+1-}B>boy=_fIIWXw^wA?YXherJ)457{`dsKZNliw?mf> z#>k+v(;X3 z%H~dY1^fc#^Ee>jovn;Q#`mrRiLf1&V*;)cPa0y{F4Q>ga2C+oCX_(1rltJ_J|N^s zU@#^_v?pw8kIjcZ;Nhyyqx3eVpaV9;-+|iH%=~Kvl4ya!vA~L?gnfaYer(^zPJDJ zKW+5Im_TsmR=Q0MJ%WTQ6TP2+bz2}1xblKTnc|9EY*e-3nS~ykq6QYYwB240x8-1} ziI;zaF2CJVtwWN`F<&yeQHx1SE1Y{}dMuh!H*OegqXD{WCj=HOdd9u^J4>^pMZsm&^GsJfp@=R2 z)dE~eWvFLVKa6^1fLOYh-0HXGvq9(I%in!YZFi#L=#emk6)VI3%?S$FIj!Hj^~ydJ;L7(Y;t>bgniQBE^Gva zkB7Nj0j1aIG5k@sp7{P!FitI_KHW`tn8ZexU#*p_LT3L!SL3c;rTbS(08Ret#$<+s z6BGDu*2tgHejXL^wX@9?|6il!sM5uqJ+b|1MYQU9=6B7|@5wsv^Wyuo!>U+3JF4zg z&+ifH(`PzQJ6GACH2JvE*a;o1trUgMF^L@&oRy*!s1Q84aJzac>dDi1Bpl9kyF3=CC zE1ooJ?}*Epcvz5vSg1x#n-b8i)Fu`GeU)?1O@UvqK(!%fAvatdovBCx0<-K(t2zb|-Grsq)Bxn(Qo3%CSEERUilHw%<5%eMVv6^xiMbX&;ja$7O~fNfEvGcnMPX58YOKwO z72yTI+v|;S$Fp7_@XNyx$HGBdXd!Dx^7?WuAB`lo-S^s>K3F4G;qJE+-9lB5r4X!? z@f2KPwIXmPg!x`)(~0%Ld(=JNk}K^~7sP9mzPwR=_*N~`TKz+4HN|3imB1GwI2(x8fmuK6>h?B!+hz?J&`~!@{pbm26OC50y4~7mNuBOd1YVh`-`}5g;D0 zRX`;&eUw;@FzCAwHRv?U%wI69ZSBhJ{zVw)TSl_63x+~AOr>^t0TzKFBqO1dc_!l1 z6-exTjgMJ`yk^A4h>+&tOTHM2;O(JZvQ*e}*(Bf=lYLf1QDssl==3fYDChILa@3oALW4CubRHd*fh4lHy-fvSB`Azn@x; zztkQxBlvUBvf2jLWfw{C9oRT))q@r+Ln2=bTw%NSxv{Ups!3?VHqE66#7(@jkVvvgcRPqGXH26{I9qyLHw0I+2+rlN`c^?N-v|Hi= zVi<1{EB(dF*`p$-=kWW(c3HV9HQsJ*WhQD&r3as7K)h*Ff|2H6O}olv@G1cDM$U0i z$EK6vL;vmi%p(%9oNB^C-LIp>1AbV9jH#eTIMho5#$^t*R>un7P*nTSV^f8wWd#kd zo`lymh0EyFeM|nvdZKGG>@{~>tNHEbE~)?FafpEuo17IbaMf>U-9e%itr?pkX{1uG_2qQzlwRk{t2*dLCMU%aEcYlm@sJ@|wO1}GzIDX~R!o#4qsj&VuNJ6AzSG+j!@&zP5!pli?Uxd#eUIP& zvrnxfGE5@i%(=sk>Rek#RBa23m2;L+O3kfv}O{{T@SzJ*UVct1HV)I_Hq=AF2s$pZoSS1O~W{| zY0J_Z0>o6G>zP?ObE(9*!l{kD(ss?msgq^y<@@IbfkDl`-c;$<_~fFsv1#n2Z{<=) zo~6DZm|qLD`#tgU54DtENgwlQd$kK8waGi;td4lfXnlx|&bHkA@U$#<@`Czn5Tz%r z|K;0OB9Uo+{KyB))6UMIu0Z`m(~TgUd9!-nrP z=fEJ}I^_4sLVJ#Msy~%&w6;EKBMn$@A9+|Qzr_2<>o#jia#O78_?g>3uz)W*Pewa! zN?_-70s{d!kd*DXWNYZ;Ji$iO23!7pVQ)(|s!8N`?(F{~@2#Vv{F%zfQ$PuUp&1&bySsaUA*8#zn}Gp_H|o9Ld*51bt-Id( zt@ZwS=a0{PnDb#i=bYX9?6Wt)vzscLAYdQ9`>d%0KeFH@U$YN^!@WB0kgf?2t>ztK z9S}+jy&RC@d1$+yv`r1EAg6GRYp%sbO+6agqVx->Jy3p@)3xzdG8qr(e?YxdB=;!M zk^_$e#*(}z4XV=z&NM~h6WBN#e+RmCds#I`3h%H^lfNXqO--DijR@LiLRY<*4NZZCKC_WzL=WPP+{_pzHBBA zog7KDY z>}1pnvUpMMi$15>p*&#c%V9yYSRXKlnk3pw-hNx0$gsWTS?w%w>MBgfOyBGb-+!sE z(P~Wrdzw0SJhb?Tn{+@LKjY>+?@EeEWqG}OI z+!Z=UgG1I=iyj|r0-`F=ZZ;cI!|K3Hz>Q7L7~-?&nBe*5h-f)~po;H$4Tb8|Tz< zX@UbvSG^c!p*LFa?3n_HN{~KKBAVO+X+348+XX@w3RUTT8f21JTglD)p{Ytd0L0^T zF|aMiZ`g8M4}WuX7)AvRcmbe69vhH|OxXx|o~h*Y4`3~Ycl+x6D6R2wXW0j<_MoVy zY?R*AR{+Mq(Bzid;N78S{FZ*WLr2pt!Q38H;K`tp;iPz){ ziUKtR&fAZ5xeHAT7svF9YLjBg+iEv-W(v+si7FGzCiFB})+RC@m7r!jPRN_?YwOrE z>YySRdhkZ*hhEDjgrR@yMW1IkEk>j|af7Llsd9##i4ZAIhfqu_@pb#+AmI%@tyI0a zP%y4OKjC)(fOgn1rYlfAy@pk2OPa8_q}FxfNXhic4qI>&6m@meQei2{R@Q1;BhR0T zfk8E}2bqi;+2-j`n;RTE?cvE9%wL13(Fk<<6MDzc0+oI4dS;4A_)f`5lx7a3x@bF_%_9@+(){@@N?0=6=J^X z%VcX93+>x9V7ggU@3ekb?&5*mZ7O(GK7C_3)=V-yqEd6n_uakrBaPG9{NYLUJIa~* z2E`n+Ew2MnMPVe1XTkw|>1t4k2{|sEoYNMz|I{9isOul>88L{DHO@c?MVlRnA5`9X zv^AS@7w%m&#@2rzMX4W|z@wNVMwVELsbFEOKehxa%wq5p%1TrkbSJ=LS@#?o8~>$K zpw(9Rxz$=2RmIABa#UVlq-L}Cp43y$FE0NB|D(z9ie$7baSdX0)eD-oYI}(5rc-H$2=S`>%~p(< z#0(F%>>n{e)B4%NjmFaj5l9PQ7~=r@AQGFwnR$r)7(Os_ltE%|>m>n)XWE2C@Jp7h zclvM7sKstictc)E6vtz}`BE#r00nm!x?^Ryn5LCdZy3^eLS$cV@AO^O+Qz%r*SUdO zZeHQ(nF%p=mfErIS0qUHl)4+q$KsU1-PM(%0bC}eCRC)N^{CBkvu23tmxtpdIIXAX z4)^4}6~3r<9;O;OZzMq>CiiP&733D{3pp&@x$?;Mn%NvLC@oT$GO5a(|KXolQ1~Xz zh{ku^W7ENH^W=dQD}XNk{$krsStFnsjiuHrby$dTk=c~fps+|mE>X9w&F6X=8Nq5! zK43ux)rDWHx!Jpw%yE6xeU11f!V^~wgmqO~njnec&dcH=G~8YFwAa_J#q4Z5t&KhN zF)2GAQaY6*v93=8=h;0z=)Ckmv@`mfU6+KZuzrCNGOFDI*v5-U3UtaA4 zWFnJnyMsBn>~Iur7$Oj%3ur-W69u%3Pxscg)$l9(#5!hx+VBhB)q5fHF|v6=dQc_& z#{!X=aAn%`Ov_hRREpwO=QK6{o(O~C^u{7#Yxxk+>hXEvSJk4g3VZQx<3-;Cz{K&f z&t;>C>zk}_!M-!LLO*W^FBgSX$c5iP9nmwR9C5>6uXn2z#aCw00C$@|rXUXjE!t?V zelEoU(R!5u9cdW&ZV6BL99al-w%;z=6A0+^FSh%zJR{P`?s_3JK{eqzmGUh9Geb)13XwH0Z+)J!81AiT5M3m>HZl$yD|2v|`q!QSW*5!==}GS=Wh z>RcW2o<)NdsWWTHYs8b~=QzwPZk{*DW%cxEvM8H=YHxAZi5+5p9nGmBheW?VVWD!M z;{aq)eQTlsfVamxt-|BfmmBM5<^r0N4p&(2Al94FUaYl&lGoC9zr>IXO&CZvRbZqE3KHu#(vH_?!a8bo|vNZ&}2J02a zGmO+zNi)KI>`?Wk3LC?F#Z7+!(@t=_hVf?+#{ER)0__hsP_oir=xILb7kejaPsIiesaMDOdq?q zewpI>Wzabcrjm&QW#63-(zSM}xfjwy#@vs>14@@{o~jUDB4=l;cfJzIoLehXy8mnr ze{8F^qSIPo)JC4eVf!Q(aC0}z6IYusNUl{Ay$0NzNxA7(4Da(oc^J#Uj-O2tKA6kiSdxtR?l0|SeV8NbjoMttrFwRur>83MpURA`2`J7SY zhnxTkAo^wz3b`N^N&&_>sWQ%@VRIVuaqn2`iLe+MwP)49+zWjrB61mwPX)eu{kYqo z^{-XC8jhD-qTE*NCO8mxH}jXng=J|*Vl-wINl6?+wH_BIMWnG2&E=`S$(t!9iqIM_ zg)cL)z2#st;9+4Y!$K>u&migA@`{gWVpnjenDu>X8TTGrDt}@zB6fdUEe2%J%ia){Rp8bVGu@d1C56k2I5t6NFci!lA#^HkZva%&Dkv}$VdaBaWAL2;5_e^= z%U9ce27p;rTa`Stm+uJ$Nl3MmG<`~H8=1ZZmKF+O!KaoXM&N5Fhy^}Dw4E#XkGipr zW5`B|th+&*JzLWiog{reveuO<_)%&7;{axeK9}Lq?ssJik$J{=2-4W{+LPH}hbg-c z++C~s`joi>vpXVvR9=j-9$Zo2yjq!mq?BPO2?F&*Cw(88!Gz!o*FB8yskGd9klX{A zS&Cr9#RU#p;u}ZhWgVT#^V_nK zD^ruYEu4YSzdIJcuiqej6MJm*iPnr(TbtsEX{l%R5&$DM(tOT6679mNH^(*2bJdx< zb&8J`Z5>qKp#4|67A3AMx}?YVmYK*wT<$CCD+_LZO%3I|*lG=w$ZUL4G#D-9`RV35 z|C^(VtYQwzA6EYNvOc|Ykt`V=UXx$cK9uGNyX@Pd+(`TVznjgHNQ3Cb{wZtBZ(jpp zdWvj#E}bD8<5kqu6APUGO1;3ak5u8dRDCUg?xmB(_Lr=r%5btUaer~#?ov4{5Q}zF zt^0dybg$W3dh>lrZRzVRon}Y0NALc+JCN6`X-~aT(_<0rkYscp%5715vME%UUCWKAog{4;sv&3B6b zT8!Vz@jtp5j!`79Kwt~aT2>}lqMf=e`_X0yNG|bI(L864QiC%sa4_A-_q&Zvu}=n} z4#bWIz)K-d&)`UWBq$WkBbWS%vyK)P=AKv_MBq5xDoGS(-oBjJIy3Jxke}}EW9^&7 zU7`?>b5MUYa((N-_@oRM7g{_zOSOdI#v6$b8dc?bn&S!HoGx)6jT!%mL8k7dK7N5+7aVKE|Pw+C$h4E!Bl30=#;%bmJHOIzt#Cqey^xGXv<47PI?M` zC@XRPXZm2@8ie$;Q8i|751Lgeqbp~|_k)eX>c(`-nIqFcdruiMkZQ+Ym2Tn`n|=za1RRe1sJu2Y z;3e)q;_%O{ag4fN2U>0j`uOUQKLI5@8xjHIT|&o z^ICEB5}UI+{q7^)pKHy-Cu<{NFaWoK$k+{c=1m=|ZDLPfw1>Z*OIq&xphGo(DvSLE zGETR7`(1Ej19S9XXnK~!T#j1Kb`sQ%esF5i#JTA=GU8-{%U1&Ecg8C)pC&o2?YR2V z-z9{1&Mg|c)lQCM*?!1vA3nMJxu^VvR&R0mmO<2fvQD_21%A3dHy2#-ZhocP$($`} zi9ps%uo_?^;UykRTZqWH*gC{JyI#&#tTN{a9ONA&sXnZ~#7?M<$Sa$0DxC{gK*-PW z>~5pygda+EF2lJjc_WFzy6Fv!2f&@SGZ|q6+0x1R`_W?MNFBH*#(fr7j*P@TS^Six zuLw1K62cuYe6-0>#vwpd``u*le35Ja_`EtwtfrzSouQnp{-*b5{=Z`ZN)?$FuTJ19 znm@_t#(u1fP34$}@;CS~bb36~hD{!{!_8R!;ZjP|gytyg5mY`M#Xg>zlXf2~rE{+= zfRce5*gfI{vt{wfEfU4rVQrr!1nE-_uSSuO1+vMg*>fpj&FfdVTn%mhiD%P<-e&>E z^oO4A1fi5g@*&@0l?IU7Fw4m=lsb&|>r*UWwcTHO*+PcN<%?x*)JI~0sM_8 zBVkbk+P6%aZ;ySy;o94$d2EUntD4eWE1JBdT!@>)fGK%+1=^~OL zBVC10O}(2sE=yKnSwjO_r{&xWft> z_8V8JCNpCZ6`RJQo6I`UQMCqp$3Z7uEt2nA)lt^B?nHyyDRoELR9BmKKix>IcUZq3 z=t2xi5luYzB$|s{+Ig_^3+B{M#A>a6UM*`(fe*=~LKgZe*YsBMhP}PYc9|A^3)owm zVQ)%{rNIK7&-ymQC3g5voJ~PbS~iSAzX$iUcjDUBlFb40$G=HMz-V4Vqv#T;LX#;L zj`7c0xH(R$w=7o8ZtUMAUyeNqy=DXE2o8tfi(VVwsWqLMJJ$@RcSNU;et0=4V)YN5 zM|G68J$OlUX6pmLz?b8%>X|DcrO*N!E=?XJ(iOX$9CwYNhPhC>dGMJi6;gkm1+7F0 zw~rS%5O-4UO6afplJh%QWsB3}3Z`Jc*e7>5aq1btNjl#S9PU22EnmX!sz65MAKOib z(`9pdJUf8?nuS>fB;g5lV8Fg z>;~)xTa$GSFU8F$-EBx+<9hRK2(nDe_`H=j(Y(q}S(D~@<3>^~m9lqr!aNa*{;wy` zy5kG&4$Zuqn$Q|KE?4Bo3l#%OI)g?U#y{d52s}-=I;os^;ZJ3-n_#MvuNm=*AaL2$ z7E1s(DFpXAjnx-u&^69bm9LeEe$c~y?bp+lE+EPwUk7)4u)MtN*BDcL_YFS|e!NXB zWU)|m%0kT=CQr!+Ah9oVZ5taS!C1b+wYs`6u8soqr&28b+XrtS^1m>0SlGz4O1SLDNs35^_V&D;iq}%2P7o=1HL*9@ zda0a=Fy~2YM(9*ph!nFW%iGfpqM%=m@5VpLo9b!05N<8?Iy?YaCD{W*^N$0Xaj ztCKAz_D$F>@_gD>P&8a#G~!jccQ0POQcQtw(I$<1?!eLW55MY_=~oEIC;Y^pDAX0f z85D2%SjlwXa-HZkc~_kD-nYVK_t-(uAO5NP%tOR-GXH(YsV^p{#vA92!gvI*J~6X7 z^Wdpz+jGl1brdmjUCAZSw#HTS&F~@*qeqvN*xr^TqDKt~PXj5Tfq4uw-5X!kn;q~+ zssrs=4og|;oS~MWNE-@^xduX=hRdHtxu|>w7nMZ+&<(?rC(I}ndys<^UJ5fuus)rh z)^6`*dgl)NtqbjOiR4zv=jpmb2^VhAdTEw8wvFrFxf)7e6a+KGG1oDY3diM8#db7E z!}%_F50F-^N0oh7S3qKGmP%dSAZMh$Q#}B;B%FirtR03`bZB`o>=><|*FS!FSN?q? z*W}2tX8sL2Z~yxhfdzbe>Dcms)JJ1J?~fUGq#R}Wz-!D|5rN%7*2}5f zec0|9BD@OX@jAkU7#zTqM76YQkn@~4M|r?hwKIAsL5m5aB4FEN9b(w>HR!I`FDnI@ zanOl%{d4Lq&;1pa)?{;6qg|ZYW)Rl4GE=cw&VQpD-lvz!`Wgcm)3-v zo!~_A!~2^Zh-|bdaoFragYxOXa1(4tRhs*#N2i4(V*T;l2@dS3cdEC;lz^2JV@N5O zGv_KrA#r`3QgR8BGg>@;)Py*lBB8_z(D^u9^`<*5-G_!t2PZIgTBk^}B0dw3XkUm) zagY@h=^LS=mF~48f|FiDou`>d4m)ZtTj?Nw*h+`aXIUw1r( z;@aqRf|EsU6M7h_Ysj$#B=mM&le=Plh~k?$B7|z!i_6~@-f{7JteU3FTFVYpU(m&b z6zWlp%}_MmOlo$_lQ|3@_c5!VprSGD8*CH^8DD7pliMoz`J+epC?PjySaZYR+&Q7* ze2vVW>U7=aWV~uuEZ-or^W!NBRy%5gdt0+D#8}tXdl5mID|}xcs#e6&Gq9`$GzCpy zEPLgV&%GWaOJh|F?Qqv;R2%eP{Xnu|Igg;hM$yMYH4)BOi~61AN>xWHoN9L*{F5f9 zoM@6)8*+8QO)GI@^qF!QPn*Pd^Cm)eVSn=K_+giYy(+faCzeZScKU_VF zecJt((W%OzAvJL8F0J^;P4&jYsYuiw&C}=h7Om#s8hn3O_LZ1emnz4|DcooZT*`hE zZa3|^{=%~c(WRHW&!GGhTf=+uA9d*U$M=2@-yv=tu<{9K$BItF2d{Z76sW;yw_an( zmli48KKCykbhhfReuV*&5^fA$6ACOSfPYXAB=UvJO+^E4M)QqW$mb6;fLf)pwa&-3 z?R!w-U&Z4h1MauN{4f$oV`~H3^#P1Ti4_(!0xKyO1oQ@cE^Y1yVV^SdUAwLvWV0$w zozt!z?Y(au89w0^azBbO%k!qi73;(UCU9nO=?#%emHNd@laB=guyQ{^2Ycx6H7-vu zg;U0oEBDNPHA;wx539dd(x)9w!d>@M)6F$YY;|*-))yDK5h4t18XUiYqUp2o7@mtB z@Kx@7OiOAEeLhhO=fbOv(T ztQgGV9D)iLCR_y0b_EC=0Ov%og|P_5Ve2gGBMn)Kw3_CNVa$UHh3v(|w$FlOh5aL_15?}sGzvo$;W}U~EL$|anZ*XA9g4Q2nhLDA%zp252PJXoS zS;L~KBP!RX6f|y|3DB1Myx>g#FmyaT2F=55l-?HS%7nUQzT%+ffihM1DEj}ZDlhgV zK!6`*)O-N{GQ57F`9H=gUR0Cn%m14(FZLw*!GF18UMuatLHQ3-y?)(|+^7DpzyJT! zEnkB65q!KV$#sk8kMg% z{Pp8U2_&`aw~dUT5)%_A_wW6A&Gw(ym^gukV*cCp6)b6xL9_k;9cZSvKGObsxay}F z{J-%r>NQlKZFzh9OK51Q+=esDU+XNI*S3Y~45i}P2Zw`KS8iX0u<9tzPCR!-_bEF~5$j;7gELOE4{;%58r?qngiG8T^4_*Gh_fh>17uN6Y zbsQu9#yD5-o&4#+;bWAjBE56en+}%R#=qHZkdRRt_)SZWax_$eUc?ht6RnqHKcslY zh05!G3I0&vD-U~nv3s`MAWTe&ui><3<2 zq1R?@^(I=+6T}61_X0w%QHGtK=3GP$2Q!k_nj;6;JF|3%6CC`U2i=Gk9=0R-MW`!f zwtcpA$>r;o8xiXLiZYuIT7w)#U{L#qwV961sEUJx= z;B^d7$S$_K(}Sn`sFYbHK&mv|$}|>FN<-td4A1s-w@U%xJG%J8N^}B!WV{~a(^h|j zvw_w1_0qeB2)9W2`o~_{CR?MyW+EC^${L<9qOce0ad@G^`dfE)qaD1kgFnRnoY497 zq(M-ES-z^-RS;LZ9=4-)?*Nxy=m;zo-4-m_J=lps*6+n3T%dP<{rXWy$TE6d`?L0$ zN(&%YcGuz7oteaFjs*cJ*G^A@g`VTut9O1={c}!YGA!3)@KQEGI1x0M#AdL9_cFS4OGG?5NA9uh2-qzC zkp8XC(SQ{FmD^-zmBsAWq;{5ivjM0zR-fD3{7Ai)wa9Gt15LM?s;xdStWtmW;*R`o z{0WXkt}~CYF@w(TgxPYI+KUk;8$AJB??gTMq*C(C@OiaXRPXe8%Q1)iiJxdDast=a zQyuJ8y1(fttE?<5Z6nz={fER&4${^1aX5i~YRVTYUiH?qkokMof+> zyMF^`5kEhWOl^=eUvzj;>g&!00iZy~3l&H_>+I=P`=cd&Zjbw{?sm90O?;$y(+}!} z(%F4JN$idnD`dKKCRDDzJW-pRag1;{%H$+jEM`chy!CfmUXlaW(O4H4hh!%g6~=|Qie(|M^f#cx`;N;kTrsw>@GMpRP@&sTm%*#sl3FjjUp zcNZ=XX>{)dV^yhhPIt!_6YjM-m1BqqJfL1JbcmS?R^9j9dCQ3x$;R>H1k7&x28#<$ zX+3C^`ujf$$gfYhDO0QbgPmYxL5RmcPs;L}u8hr;a7#_$u`o=i{##uu$fLG;FP#E7 zU8U>PD>9KkyY|TVw2J&0nFXzgq! zojFxHfH6Z%j&F|xkUCI=>#L(;;|y@cV+Ct&(X>NGWO?3=9H{3Liwj}&1;a?aEij^+UivDWO#Vj;k|!ct4_w*8Wr1J%jr7> zuMYU}tZOnKS2Jm=7lNSA2Q5+P9@CPa?1dAUDWJzrAC~q;P`-F`I2a{O8-at%<$}t; zN)GsvonB(Nlt65c+Lror`0ZMv?qZw9BKZkZzOIhD$qFaeCD*g5p>SQa#m3Pwb$KdW z$Vet3YaoZl6HFRc&mkTbht1Eor}RjQCk&$2}eZ5jPAwNVpcavgEP&prL>!#FS6B~h0|?LLB-yM7KN_R5e`n5+RZ>9`U4gM<4 z)fAdkV)Yo{g#E@B%0szfxeT9c9C&f@8bdzFNSY(+VS+53wFkCwuyNnmordE^t2pFc zpvVnB+AfI{_H{&zB9XkcX9!QYo^)+E^~y@f3s7qNVmU2$LAD&9;nt{sZR;K0mri84 z;~mOQDCDePC^WfG_D{5%W$**&_I0DFha!)qZo$3w9#6!I+ie_{Mzgxs}BBU77shdpX0&E4@N7O9uL)U@IAHq+{rfLt<(1 z_?|HadJ2a&*b5r_SxxJK00uodN4*d0m<#{jq^m(E+-~}E&$Wre9l$gv3>}4`&vT4+ z_F)b%L}F{8zsGj5AR`;)^)h+xu0-c8CoShM+d&M{sKTI&j&3&}Xvq4l9QLMm=!35t zV#l+v!9iYk_ylS9iGiYNMmziH_D&>5IYoWqg@@;fY~i4|aO?Yh+Lq}pndY4Le335I z>Dc4j=Ul^KdETAQEp{KN=Xf;c!23%^+RZ^6vu0kR>*Uj+xRX;Wxlfl)*H}eP+O77k z>@BE>yb%xNZl6CyLU02G53l~)hv3IPm=NCM@vFX~+dk*JmqyFZZbz@g*;I)28CHn?IuKuciwH zNwTZ{_)t#<+t~Fn_R_{5z3e8JIXB^?^zlhTNdK1Z>rZmqNMl_&?cI)aGfK1}db5oY zq9;WDmXj#9Powwm*A(!dI+I-5#oa+8t)<>QJshdQRYeijck6)Xf zVNRY3iBC)hV$sUS--BcF}aDO%hmhu^Leg<%^2u`L&?G^?q~*Sv8>hNURGx*C2$C=b+)Ma$n<{ zJs?o0WxQxr;e3-oEX!Yw$jmo_>E?PRNa57UP{cp%=K6fi7xQtNST4FaU%aaA6quD` zqM3>?#`SEcpz>-eO)S(>Jh;(8q=ZHr@g1y@i%2@&<2~hI0%`J@Xr_ ziR^>`iN}biA-6zotC}v(0;?Z`uopYr`xD{cWtNQFfu+$Lf(hF?>8A@CoWed|x-qA+ zsEe-j(IA=B+hfzA-b;6K{_K&`gsFDL$oSwx$=}fihTlfY-x~EecRXuJ!;Pov(pQU; zAZFei5&jG~9mC&jzTpO=!8w7ZzD?Ml>gOt%eOicnGRYFo;E)t2TS(|pYySc&ddjWm zNl(U36)s<2E*4Q4yjG<_SP(o#WtFfadS3t=Sk5jV1XLcdm^K{?E5>PlI?+7t0KZ}# zG*fc5t7#HR&P~(ml8P7_@GyJAooeR|EtEftxA$Sakx#JGaOFfgbm^LqnyHP>^yLg0 z^X)!Wfs02$3|3Fb4DBP#;Tev_(&FrB5JLj@-*hB@!{#l zoBoIB>z{qa#Awph2QU&At@u_Bz-R5Tv-Jf`NYmmD-u&?zkI7MoxU0exj<5EZpa2H2?h8HCuxnVne+RsLC)KYV)iQhfO)+D*fn z5aVX!@uKuL(y7t&bqxpo+2omlVC$=qn$?_#bP9ZsjD`H``EoB2A$s};n0G9HqpItb-=IVjhq+W8tjgc)Ub-vk z-1;hF?+xnE{yP?cglcFqRAatRW{4yl`PBQM2KTc>LJ=Yy>oEP^GcX`dZ8+rg3Wi*) zLvp_h?Z;(VWqZrvzOe&z1g%tD%7sy!ErE06fWIrtP<2kjp)fsC6b?B!{AwIZ(LT6f z=SqLb1{Xqlu~N!eSH#Y~qt_fs(OkcEB`;x)fDd>56!YwvaBV8S`x_h+)ggr%*`zPO zT5U!xWwQt6Or;1BdETAZ%@9U<*oJGh1|WJGXW4jq^;n@_08O0_;fjql*$w1+&~&@M zBhVJ=xy$#g;o>!cY9bq2se%Y3LdPp%Vk&X{SZwiLFwjOqNs@;&f8kg7Nx%Z8y=-#Y zPG4?DzI&;FVrzHB+ShOR8{{NHqsc_sW9ZjeJ#^(rg-K*z;nm0@D92`=WC}v4Agbv* zbQo^3ILikT|Dp$w2wdD6T-XRbL;3lfsD-rsgmH18ev0HvU6~tEQP_s>t9(9>*^!23 zj7)+0Wk(w_tQCl1;)(MEjMuNqqB(IR&NEsRPCPfxlW~VCk6E!I2kJ0Gusy~j4665w zD5)zM{OEIgJ{9J2V)%y8uj9w&uMS+{>5iXr%%6Hbct$`9nPW1#W*QHRsYP3WdvM&o z73#uA+H*4_Jkx#`OEf2X8Q=Ja(Azp%nnU=&t~o8fvrC3nQ!>02$2Y`y%zUf8bIQ%5 z;t>fk?m*LrBZJaxl|clqn-@`k=vVS2L!izp+EUu|Yzh64Oxa&eygu5A50ovEy^$=5 z6Jd5(;dYzk#^LLMd01#+upyy%dsOl5)W%DlW0DXzQ;5Ly>P#Eg?B%yODv5H)YTKAX zyzwD&%M7Qd?5c6^nu$5z)Ww=tu{4*rRQOP8BbG*38&9I@(aSh)nYZP%9cdxTUWwct ziOK?*8n-~rw}A-+<+6JC3Ok!=1L=pNV5NL!1lxBb*G~oLK>rY4Vlm)kXCgQkY%DH9 zf9XTk-Kb?J`1l5v^(!MCx0324OGV}N85p7q=;@Y$MIi>Z*S|ETbJ`7^nAtxSNCrjG z0^(9V2g1^}hatlr$_)Hbyc_`W{PK_iPu4(a_Pm>W11rH1&*{P`)EawYejjO46}F)d zUD4B?Knyrh;@4?>gv@`MmO-* zslodCiE*^wdMJaoG&}zMuWwKMKu-0LPlRw{E6A#HVzD#KWQi`Q75<>Jaal*)V-w(dS z*yTDgc9Ic5z-iNfJCuvHyE`9Y9`fUPEB#%b+QA6cs09rZ9TtCV9Q1jLHhr$;eao(P z^F4MBdo=-E%Nx>@Q=qA=9dQP3^0ep9q>9-qIF~&0&PAml@B(7UPKo=dH=0Cr9^K2q{zz{e7JNtq!03Y z-bULS*HSIV>`t8`vdfRBJ(qY{Ja0e9MQ-x)(Cb0yWOwCJSYnqk77^#wWK+IM!@+_M zc2?2=O_ru4fBFY57nQeGO{#hUL$u~7?>HoelS23ipk6d9{4y7cUliGmFq# z@`c_7pXFZr72?*@O=`#9OITlWbU=+CN5Pr>urH{_<;EJj_zi<{eHf&UKo6d zR;vVT9h0#~{fC7{8NNe}oNHTNZu=o|Ywr3wN)I7#3{?ZGU!-!5QNc0-_lmD+lrjn2 zoE4b^;gn6+c}#voW$rVNRK>P*()~#r&%|)I$l~4M&tXCEdSqO%Jxa8)t*gQlc53~y z$XKsN0ieDZYz1B_M+v@$43Q7Q% zOOHE~UG{-qRi(#)n?_tD4lSkta}xJ_g|QGVt1J6_9R;f?Z{~ z_F?zrA(1bY!uB^~9EvcrcF4%r#rIOeP#%}eG#L)6(y)zRGscJ+ia`khrN_1#(mPDU zmrq^p&5l~utQzk5K&_mS!D337>{=EFc!q@U^L1h~_^a~dGqZgrR)VGrvI{Q zeFJvz=Sgb@+D`|~vV#id#FR&mP~MRea~G>g=K3};Zaw*E zTX!3Wec$xlUe5r~tvxxr06yPKm(r|emd7wE4L?1@LqWC#)ThtL-sld>*a@ksw;lwK zEjzF$l~aGj71q^GJ(6n7`Vdv|aiX{-)TOxL2Q5ZvL8PAIQ;)1Nd8**CD|h4ppTfK} z+uaPN6NZM!!kdqcd1fMZ)lGW^Idb8uy`y4rc8A%1xH;ZmM+&2JdxX|lC5Wt6613leyHjxX)3Zj_-^&7u!DDhy}%Zic*Ku&$m;6G z0iJ1(*F3s(7RY-)m~H)uotq`DC}Kq z0q?y%tbk;zG_b;7Sis8EsX^kog$kPsPJSo^CIpUm_9;$rt%A!VcUrS(^7@MWzt z3ERt?(wWcRT*8`W1=r?=uC>gU?6tSLWBFi$i*y5GmHPB_-69DBv@dJROK$UT>#qR- zo=gA!cm|z!guj2kE%g=iZBbKmC=mVep{z)%>}ZrEeH?Qi*~M=tA)CH9*E@R&Qg){I zY1M+rV7i|vuIPVzQ25A848sIQlM=OeX<7lq4pUXQ3pP6c?v-4{e)ju87n)Zq(Zw?M zZ`Mu76GJ76R1@%vSE z&VL6bFj=oGad*1L{*?5}Ft9u??+qc-=Fd`B+e(DRXk8SI#ST5xdH)L2X6EXf>0rvw z+9h`d5yW{VjfHWqdK{CM>docX0^e3)Qa-m}{Fvwp3K?1H(YAKciB>(TgE@kN3X^_w zr`{`ntl#rO!R`mk=(|5>NunB?=5}?339RxBel69w?8QJCJ-&NZD`UO?Z4N;irKrgG zxghpdCa#~2sKE+Mc{HIcEWCJna-s^wNV1wLStwAduejWw$+gkq>0KCfz z1MgB(QaBdUlFN&XdH*1B&0mDG|Af&3c%!DK#+|395WT*;YiT2S1&{xvt=;6jR9DCM z`Hh~g@S?Dzv-4*!?WQI{p<*K+pOzUAGEYS}1fr9E^dUuCM`vU0AjZN56BF#j{;C~5 zEhS&^W~=gJMy3&Vj=dT6|1-*0?tOQ6>+S3mKCG7dgZuWM3=CE@TULRC$wJEI<>j+@ z4a&+(iOf=#Ga29FORuz9sLX%%=Dj(Xuhzoa9E=VPt@IB_Hb-=PBt^^?&bA1tsFi3d z4k9y6P{ed+K*6Qvojrq=WAOJZ+Errh?Q#`Os0~39BU*^;TotKSt)X>!Fxa;@~VciN}Ij%A)|b_5=%8?Iwsa zzOk_}@Ts>(wdG_aO*j@wHXgO;c$EfI9Fv+n)ldSRD{n11L^y?A?v)X}<$1!%lL9RX z0!T#ppdT9Y74P2i>BvgY!1eFzuUM2R{?!p9X=+fWt@C#k`g$ zhn9Q`cf>tD*S20zQ*>g3PWHzod_V7hqR&vOTVtFoA8%7k-Zi#@|9Ogxqn5Z-nAmX;Q3w$tVv ziKc|~WPPFZQDI5my;U0iNbA@8n5?B-kJyX8~`sF`aN+2L&&Dtnuns zSan#>IEyljPyOjB4^q22Yq%oNu5M-j#OZ-^ssd)2!5hWCWPb2s<4Uh^eqLUY7UODI zEv?upIQ&{gYOwt-Q&TgRpFonTcA?cVnHTk_LSqOK;bSJ?UF zt;JHMl7&15Yb<?G1LFs-_hz(Rb4RqRLpu@}@e zM;WIs(rS;X>*yOpltK2BDvy!U-5z{VPl^PbW+{i>#w$DIJfV>AyEo4l`q7_xPKa`I zJ2f{qyN&NW4_;^P+w!3Y!_h_-`}T1$i0cM&E};>1bM7PjsK5RaSzQx%99t)*^3F^y z-V3w-jJ{oJJ6DPe6@vBk^-(4;aXbp1_rvsvqRG;yH%69q2w)NQdu)Ri>$S;8AA?t( zI}uAgc+pnXwoh&OQRobEab;uU?$&lvR)qKKEk>AD;`vwtH&wy4x8nw5CWWyo1s5$W zV&A?qEABB|54TbX1NH;P62WdC*K*(B>v*R0L>CADmY)6w4THx8w)T(}dgu-TJ7B3g zk5thT@>XX$imk|!LE(IC%tB4fc`jbwjP;GBvE!qa-UZuXRrm74_gI|CQg&+O9xr0~ z^XrYn37&hsnK)Uu?ocoa!ei@p;VcBW{fS@wqd0ga&!2aa*z+q3-=+n|962SXudk1F z8yVjG2ol_=25+ncN&jw?FlVTK3Dax?|*pt_Jf)L;a2L>u3RYVHnDKL5)w{+XxX_XKgH^|9;6V=jP#L2`iC~ zAdq^KfVT_$(O+{pmuuiJ@R)|N4Oem%cDULt7hX(u9f+7nA~eP%)@(lHQ(8ld=$_2* z%mUfIwYgciBDHnJU+AL9tbm;7=1f8xMfbaVRNbx|KEU2;bzX>3< zy*IP#E9_7wR8NFsBr^s2&FnnPz*z_Tw{|4tnMKu?2y3eop@ZF-N{-CnR&@a#MEd$f zTE$v(FddP9@JjmA?Z%WO(t|E7>(F|~&@0-EClpa>Q6+LuU|^uxQ27$6Wo3n1E^zm{ z?n&-@9#8CKTvnA)$4(Y4a|)DsZm$#~{5QWDqV1}+;G$+mF%+WT^!#NB=KWI;gu4@X z_MU!z$d2#kU3M$dmpiV#dWkVYEV7$+|focxyY`-16=y`pQZV5VU}!`4)sx}4+aO+MxD z>k!VE+u9)5ZJd?7%LR&V&*d`0H7~~1c%55|1CVt;OA0MB7 z1PIz8OQo-;=)xDJrM2iMLQ9_0ov3YvTeF(h0om^w9Q70;s=a@EeuS%k=+8BF%;htC zaHIYim9B-w#hko6A(&CrI&H15q*f_l=NaW^Qi>35l!<+tnMp*=>}Rekf!pNITy3@P-B+ zmHTfPI#Hs;{n-0`Kgqe8+X1uVkyFao>^wM=l^c9yyon!9nvd zu=Xv>e3HYNE9MQP?Y6=WInPh`VMY(=hvc09isn)ImTy(6P&@_NZPeZ>TFJSYnz7#1 zS>o48QVtuthh(@;np*)Rl;yb{+nWh$-aVJq&t<{8%~YqLEX89Yu)5`OHfmMKGJCJt z$PKSrkaQl63eJx<+#nP>i3DOJWsFFBZ{=ZMgjvg^8=cNpbhSB}g z&-Xn|PDpSKt9PNFAfk^Ijo})M8rAd{8gswQllNFD3%=YEyEi-3x2^4OXi%mO_^#2V z_6W~l#zOfvam<-BO=|`%>y|905Akpbn)}eNq~|p?I~SDg*n?-_LBT;3Iv0YH&z(gW zTsg+*`gp@)snMOOCFeylK}fPdabVW(hG|50q{=jusd(-P+0-Q!T}*80XlsK#evD8Qg&6Rs{9>eAh!dkZzByC@%g8i5j?%G(tompJW+uYo= zR<%ngXxM15#0FpWr~W*9S$NyXz)nUJDmKtNq1f@$mo3lh?Fzy~Y0H?G8>3;`bYT(U zh1?91e2XS<#V=V!uw=ovb89uwJ)A9c~Xy*m#x=ii(JYqI*tudaFWG-bCAlE*?; zb-)<+%jjt12{pV-r$wc+VbH@oqJ5cp<=rU}8+>}n35j%Af>z2JVT?=Ed}Hq(>tHPo zhjdAN#V!pS-kOmQ+w60H{wi1-8VGJFk$Lr|39ag$$e0Yqmv4K{>zXotl!1#er!oRa zT}f@n)yg2A z+(a|ZkBai*T2a;jG!;|WXhRgX!E_Zm!*HlBTbT-Sy^ypF%9y5xvMtTh zI$nXPKVsSNcHb)5E`mxO3mHH;9EBNOy?TqvAaAP^%-k9W-+?URxTQ^-%txS>33y_G z|FAJQsMLH{}T<*cqtX3h9Q8LB(8Us74l`9n0N9~-5IIIO*~(4?DDO^IgkG8X#dW@t1C z7|Im;5(USH2W3SmRgvS*X%eX`h11j1c?rVWu8ekv+EUfRIm8v_)2cA+4J&zc2jgw* zJHbZBSg1;yrqR%xj@?Fanh1kr(PYM!8wTZ)xJLGfSbF~0C!k?mJ8X`2n510_Gs+Bh z+1RM@nR#h#{Z`J2q{G~_?kTvjT{ZmPyTX=N{^ko941%&Eua34E^k9QZhidKJ>NIQu zFvs}#B+j2dk98fozTY-f&y9>GzbD(8Gh- zF#`aH`_Hs<#XR(VSg^5iB~a36Suey+ij(K{62FMT7MK%oaIb^k-SSmbR(1nnN6C-C zK4ixyJs2F>_ie8pbq3Q=5V5O>!7Z&C#-Du-R-U+&zCIB0HZ44cD{mphEs8AV+-_@G zVXUe=jic5*D#Gv}TTM?+qOL!;M$=~k@wcK+U>i*1&)!-!!Qe8iBB3}aq90_Y8lsXD zvi~QaaYbH)~2oxsf;0aDPD>ET3Y{4NoN>Syq~-9LnrF1lW5TcU;=%S+~Z` z`NW8b69)hZqtIZx8(h`=3rNN`VNdO<{S{5glyXh7t1|bgd?VqC2I-sED#)FSa#Alk zU#&7$&Gu%!X6kd>X0-R`>7+?-(W)oV?5{J3B^Ea#SggQE8VEaJ3k^`66H7*(WXM;I z50>lQk-&P4SjMT1TX9yoB2zeJ$+@by!jK9g{}nK3tzu7QLAa;w1XuGrkL%DzuwCoN z78S-j-aTBY4Dqn8dBEyS=`k%C02mwvK3S4_u)VW^p3lFA znXl02E>xyZnIhD^SC6Z1>r9OJoO7}&pCNa~SGtU8d01BW_cZRHZfUFaRrGN7eqvS- zH+QKDe+Vcz=23pORaY)DK_)EBnpaRND)swq>+&or<`!bAH7FfjRJ28)^8_eh-=gjI zHZpd1x08gl3nsKnyL@Bfs`U$|((rnq(ysLE(t^N#!f})8()Fbgj8x-})&~2-YreCR z24KOuos$>aJ0*X!M|DIW*D#(NSfFRt6v-friHQgc!!Zr(kth>nNnx;h{EW&N&YuWs zM`D?;0yg}T#FSg=;rcIQV_jPtA3j)W>FN@ZghCnX9@~WQZ3)`1IH#_h67Bi6uMcY_jdDI#awr_+FsZ1xgec5c4f%WPb8EJRBfg!zWkgg#9AL& zhC&nIWZ+mW2i>eorhA`va+|f)b;)ci8yjXJY#0oJ}E}WV8L)g#j>LXGK>pE4zU60I`wl%)JMFk1Plkw7>7b~p2 z9u*}f5 zxtp60-3Y{cuSwWRK0)bGo!_KD6t*x#O z^qL*)AXh)T{>5?k#chCIi_H=V_=}5^i{c=NQ*}GVOzSlpS-VmGJJk~rsm=EcLGYX2 z+-Xl1bLsfvZ`ru+8&>9+-(X&0Bq`Yq%AY$id@`#kab5g_6t~$6_j2q9ruxqNZMOFL zjcBK=br5uW@p$~q(o*T*;L(MuK=%nfT)=YQc-S_vEDUFrrCv-P0!YMvX}C$rv&`IY zR_DzQi_qfY82h+~qiw2fm9GaPZ_yo(oRYilrO$(L=rFsmG!a6-ZFKKm0%KX7wXIqnT3flC ztEUy=mGR`9RYW2pBA_uqnH8tdf8JB#4$?CkZiFjP`dGhv4Apci?d2V|7lr}mus>H-09NgV zG!mYfHN82(rd9>dddNam%IePIP5h3GA)32&5}}WkZE0XOo?tIRui&x*vviUPe1Nw8w%8XDu*7stTOJ3kf{IIR2T zM&MFEvDgeEFlbxBaeo{;hTb^#ZGFAUHf}e7{_R`uTrp`Lq+$x)&+B#|L_L@+o9^O?@-!B zkh3B)S_0>LO$9pN3M(imU_;*p74#fE%9$EXXE@q?{q(%TLw0miNzQGccOnAOU!q7~ zyHlL9b&sV1bccm(4i`H_2DuiOk)fg*ZPCaL$C&Ci*@oCa6?Ic2`*EH+?TB4wDs&SP zpqUuic=@`%+RB^uBuS61rmitv;iWpUvAya9KS^n$=+sbAh3d1S#s#mcmVbeRz_}`J zssk$JguEf&IVxbA0LhIyoLEp4&I$%YQX?ki&gb$Qge>l|rqU5e28 z<2M6CL%^T0{L~c{6nV`Rbi0FoHq$!Bj!*Oa}l531_a($b}6nA1SfXPN;OCroN1eBTot=oL3U` zOM2Y;a?)nIw=Om9RrfHK3!}M@txXu(`+dFs)^(t$bVwjfu#i_T`5_B?H`s~d$70Ue z9r?8cU41?lXR99-U|HdbKZsW`0fm|YI$ciIWzsx`SKUI2<7CP7^l>6a;69!sTo1FM z)2J34)Ov~iw2BEX(9>9_Q#~D|RM*iOFoI!pYcGJb81v)0%X*oT+ODXqa7vpBF=J?D z{|AsAoElHGH>>t~t~XAElk1sBxsK7}mKtzR3h>(jDeAcIQSWpVq50X*+TAz@380@c z%7qhTSu9QlG1kDtGJ=}=N4e(RuTBKM^NRv@6^rqlYT3|5gFjOf)rxJ#qF0 z>-$MsV}2K9?#DoK<`0n*D4V=hCAb2$vSE_LmE)uz+UR$6Vx~j&Dm0b9|12K6caed7~l?ReZ3dV!EPqKi;DFA7~%8h-F7}# z`$rH?Kmr!2#3r>EDAdlXd;NltqFp)`vatCgv6e>}#PPa-<@|RCb=N*U`uH;TMQw~G z2lPvWehlk-dq^RcQs*bUdo1O6(rT;C2VzdG3NBhav%d@LMqYAn0c8NzsYi{mOx9fT^n^5NfbAJTt9DclnEVXmlC-`JPA>^0 zPq2${u8ke^JZbQ={%6P|=uZyO|6PwxqYW|#=~{YvDNc2lrEY0xI4B3umPs_*Ebtl$ z99oqyoUGzI2Yq%_^!pNe^t2poChXbaDqcaWJtbqbA38R;|Lvt(n3F0$7$1!6B3o7&2z5It4D8X+;W*K9kaaTWm;tqQ1THD z&m87aZYW{8Zm%2I)cV|#b0f6|=$wd_v}?$LTok(0L)Zs>e;hcZ5FZ`wTYwFEcpN`t z7p#@MhMUn@y7`aO1DHP6fdE5t#P|=%NXz^%bo`h+4-E7f;nsJ*es$gJZGy1G`)sLV z(=Hyx*#8?^F((-9J4Zlpo}t;OjD&4kDCPRb8;1IHv#m-4hspOid@`ilZLoYPee1KU zRcicWRVPPGYv)Pm_+kfkqD5NWjt{zN(9eg*|I>oXzf#46xfij~y>;h~9{1jT)t*wL znYn+vmE18wa@fbF^X$9c4@Tg~Xrz5K!+7v6te>eJuLaX3RY+(I)`+q_O8X~#UVR+> zpK?X*qXeS6%};ECod4vLcOR11k?-a=f1ysH9G+YO(fKvzU&s7WW14Lj+qM*U-n*uF z73pEO2kV5`9<^N=sr!B#z{(2(S&ZQx{d>82Uc*O>?|whFs}27r37_wuL*_}7yY9zf)6^L+YB?aJY0!NDJ}Cnj zEES#~e5XzwXk;lu|20uKh1r^zRAxoEyau)hLo%R{c0RXR$3QxR4Q|88;HgOP-4Aze zwR}E43WQtFp}(ihtrpGs>4h7Xf%hLnN&nr|2Y= zUHn=fcJ2J7nsHj0O>pBsNsaeS#tVTn4FG~4=;FgQ5dV-SNFzuTZ!(`RwF#~_(`Rlz z>}Y9$^rTW7TtKnbO#5(!6QasvND$Pp{!0Ufgo2)4PW7FlI)BQ2#gz1P6ev-VzJ;*~ zc@TZ>5!DTdnB?fA=Tr7SQWc{aU+Y`>ozw#F_<3RI#5s3!!{E8IxzN)1khiG|7+5g) zhK7c9eisdG>Qqst9!k#LqtsMc5Z&WbQwteog@uLPw=^iBi$UP7bjXaoNj-Zmx128; z94{;+GyavOs+kw|u>}#40RnI_UZL~nk*3chDVEiZKL-Bq^kWjbb1V6RcPfyVa}^@@ zG%IF130e4UA=rs%;ig^iKLRa?Z$I0h!hb`JJ)2@~03|gW(D} zl7L+*52!yg)6+?JBhH^M)Ty3x6c-zcf#B&QiNDj%z+*3tc97@TDda?<+PXSR-ai@| zgpYG`dyOebO6qa7r>4eL$ALu7CBQ8O7;@?x1j4Jz{zUYxl@)LD9PYRCz>x)V<$X)5 z>z{;yL+HI)KtiH;JJUTL><#nu;^@#nsSp_a7FWV~q4aYWLfzj3d#Tv$*E5`PIaQK3 z{`d|pKiqx({J9(RWsjWKx=!<#FRs8HrW@u|E`)T5J^@*-K*|36Y;H~^FbGP#QPBQ> zn;qL$56<)R-|}LnW*G^yJQWj_fB(mtw4Q#-KkZN&WK{ZYh>Z9Vp*T=hWI$$phGD6L zw{E>WZaSf+4Skb#YJFzb zzP@sSR)zOs(<#Dn>3|%KxKkM5*OK+#IXQCr3b+|;(@#KE^Gsz7d+CQMG}G?8?X%#w z6XB-7YFaTCg~*oy7QN>4bH57y?NRHtmHAi2a{mq{2+HUGcr_cWx5LEgbmD(hBBxLO zpMm>-ys6-SG&uivcj`YG5{&)-KQH=UXroh5N%A^1R@~q-Q%2NklHZ?W@f?UUR;dTr zKJZ2VjW({T4L)!$3qPVP4pWM_Xmkg9}!Nxl&G7qf*pPBixDaZ>48TgLwKPaXo@`WV4?~%;{C{0nx>+s}5# z{>Ma^p8xzJQPjHTwNfDE0!SfH*#d+gbW)OuRqFSOEiLCeDiPG|>$|+VwpOmTzP4%t z*s1Q8tm?aFGsQK?TSobnV|>==C1J-%UWLG5z zP!=~*rR@5E@J=jXJhiNGegXWZt>msa|p0xz?@ z7W-l7{@{6>{_Va0VUFkj8?*8Q+<(XM$9{yNfF@hdl=%z)qy|tW{qxuFSFQH{0PZ96 z^71YKzkGvw=*?6~n{t1}fVjWEv1X@rQLjC-{uOR4E$y;CC~@|rV$M}4bN=|^NWb|3 z0AQ-g$MD;tXD8qKZ};n`AcN6$3%e%3#0jiYU6)$^i<|tFzdO+D)(lcLkDna4jo)4) zLkb`I98R&ESIl`mfm}TihKUjgGV!O3VQlqP&=cW%=gjCMpJ?6UeshvoPcjVDXylA@ z8cxjF=eZ@9+rp)B1d@fR@9kQ3TTKh6^#pD=@j!b8pCmbvA$`Z#@@~&S+m={FFYtj@ z&wgtWh;Xz#NtDBt@F5~h>-c}>YR>w4Rp`020MNFq`Oz)H{*(xXx!V?OqtqV z7kKYS8}CQ}+q-#g=YPJbo#==b2O3wowU$oht5XVn(sz1)i*CMG1`vrAMb6!o>E1k- z^7*q+2T<3`*CLkkfAw`z&$?Y%kMa-BD_!cC*GHDAO}j5r1qqAM0>TWKpwlS&m1wk2hq zbFH48zmwlI-y7*X?VULmTMtYS_gseSlO32&ViGVIECuG(XumKvm-cIqeaxvI=*{Qq z*0+D8vA^VN2)u&8MkRqB@h^|H`!l-2q||iek5gO@iVx?lz)rSR6-yz2f6os_3u^oL zyc8B!88xc4^ymOX0bTXt5N@Ks-ofgI?|lVF?W#^OY3X+9h>tBKMGgT)*V;yxqtLrX z>cvw=5{D>qRPi{q-6Vf^VC+i)2H{ibPdlZ17WJ&0hPvX%R+P4WT+}E&ZF%0U^In;U zV{A>gr=~6N_ov*&;9X$4x>&HATL*wz7Ei2H-rVINy;%X*9ix~7+|S;J#i2FC^xSKw;z(B@+2M+sc`Fj-@Ph^IR+H4 zbS$-UcH*?Ty<&$X^!k_6`+l%+t!5$XOJvoJ(Oh5vY4h6|E}WDe;rCv8d$3VjM<8so zLGLun6gWt&rG7nGGB7Y;-4j3dMpg9B`qERt!LQ`KrpzAZ@cLkbifjv$2Dg>Q0WT^K zue27XQSrKmvR?{t@9S(R5PqD`Qj&j4<|scVjbw3c+MnG`e514S`!(>$lvM1rip?S- zWH*!V@~q5KMUCtHBE3oG8j219F{emx&b|AQgX|y!tDKujkI(3T%^^yD^UA>gX!L;a z1thfhSr6+|!#BTbf+}hz7@87>*QU9+xX_rYC@rn+pYOGB&=?@1P$rvyj_S~Ew{lEa zdyuL(ygMU88^YRbD7Xqn>={MQ3RZN=KtU?NC#2rd(&<1vw_E2G8T4i39cdYYQiq#u zTU#5?&T26Qs3M^Lx=aW&w_lhMkpXYf{J}F{Z2)t(`x~-0Dd{+E2&WJ1mT({*_%TN~)CDt8pL(*0K)NF~*zn%)eQhzDHs(SVS03WSSdhUTXy)nQ zt0?`~b=ZH=O)~SA!Ykc>kmCFPOMi#1qobvL?4xODSxy%}(@vc>{VdT`<%AG;H+YnL z8c5NcPCA781JMI{7TTFg{+3hMHDC!SQ)J2ITurER0yR;Dl=om7$*4cH0~5bC>m2TK zh{35p5oy%x*VszYm!WPT#Y@U*{NKy7j-9Oy;8&Tesmq+w2y-#3_TkE*GF^r z6rx^pN5!4)Vq}zPtmB9J^;6x(wqoiO1cAkq)c4H$Otq|j_<#@Yw8~m$x~yCN0Rbrz z=woRQnAq91^#-|YOO0Mn8Nk``DftrtH-NU+jN>x{l+W>jcQ0&DUNY~ye3?`RE~)Q^ z4Zq#K%pL@`_cw5@tELEZKzYg$=Ba|w~4W%{j~D0sb&k3Np7$lUcp~Wjqk3; zfwby)aytflN}3`e=Wf0S%mPE?1rQ|gE2P_U?xWT*PpzQIt)-)rloap^3_l9sCX#wg zdgc3WI97uXnrv8#tU*inSS78PFChpKxyNuIX`L^dR8}0sINP$RIlubtK4p zdcMrM(ZcQiLPxNVGY9%ZhBo5nwFD2l|Ke`m zfS4Q73|LystZ;G+?0lOUf-n%*Psq&l=#TKA^;=8jHi~_Dg(#{IQuUkewe_bAUJKji zGy!I#oSa-D-pLMnC5mIwR82}MOv&|aUY?!T21VxTo*UV;g=5+5@eWhYaq;tg)_8r8 zwpgPr>%+SHUQk3pRTc6}(mBo8mhm1=x-6YlG}2yVUvZu3(@tC3H4htYM2P4!tv{YQ zGh7>%C||l~OZFP=P3mcsMI*$_Br{bRxCQUUZ)(O?QyT1k>jTS5E7Z%3Y%wAHlDp9>qtT@;Cul36YolON>+01< z?hY=AbVwk7k^e% zb|Po*$I0C{P-h5S85l_UeYVRk@52XWPP@zOr@Tw?b{e1kBLtMIJO?J6^b2a1@Ca^A zO-=tzJ7w1UfH8Xyfe*RaY*3jaXcG>yt%R)?`UqDsQgb={$kC$_cQUu5ESTW*P{xfw zJS)z3vFp|;0}}d|VqF{uv?zoQ{QNUzY(uGNv=}aXr}w{P71>~IUaNU_FGY0ogcY48MS~WdHC;sYL5DnnLwNfFSHFi=sf>kA6|7` z80)do@Y7E}38W($SceuWkc5HN7?P+}b^_!jWF6u_InoZIOmkzamIoQVwD7^&45jcv zc-R(ul2E2nmcUx(jHaFBxW!KcynAq>R&M2f`_KjT^ zxKcEW9#2SGbE9~wZp{<5hy#-h5@;%zFtSkqW{h*1F+{gy{ESU;Db&M6srqUEa%-1F zp^z}|Phx16qb!eH_@oocJU?P@L2_v7?3)pyU2-z0u3m4vO9+RAmYyq5!SSt{0$yE3Xx zmWZNC559fdkI0SYmO^^x9`x~hV9;@*>p19)9jRskE4<6lh?61RQFnU&uXn)l?F{M( zFiDaSjg7#iG>~&kM)AshXkq9q@7W=AGYd4>6FKdZ-@GYGmqUS_sEdLbuE-_2UBvd7 z-=W7zTbFk+rz&PaSD>XU*b3A5^z1Cy(#&?~NHTGkrB@f!p>Sh0v%kvdUbmB00*YKx zeBfwmks%>v6*+R-s+x8QJEr`zsp|yaNZ?|qpgQou-u73A5R~ia`^P>tHLa>fCYx=gi8?4N5MQ#E^ukYPI&_Y-7AxYJzVC6Ko}QSU;s z-&()MNmDp`_PPy@(H|w?*PG2!Z@Y8Jd1N#e0r7BTwC|`z^~jQREeISfGKJ-4um&TDFDtuX5Qz`G5HRl5$t3 z>J1sE9%7%py1MK-u<%?gzG2FzT(3V?7hsAAe|le*$DguxDu!SAT7{zyfr2q}Ug_sj z^54=yu|k>6$Ke1nhlOnP&g(~s0gqEAZrvawaGdlQk29&f{X2v2(qQFob9CyoadG52 zZM<|?4J-R_{@d|wK3Tt-)fsyK$1rsyY)XS?tL;zODZLI5Uj}l5j?q{yQSLN%?0pML z6Md9A*wX?G`+i+%ayDWOTb3P-Tb6DS=4ds19hJ~$uY0R#U}t^%us^dXQ6yF%x3%@Q zKeMKAxSn3MzFTS2Q0?V5quva>+bO_4s$S_Zr36}z3=zKyAi<-0qGqLqgl-0@t~yWE zEepe%G-8eV&OXO_bOdur7!CRvvX;gK*?BK(RXB!bBLX~$z@ml??%F_p8EdeyVo-*3 z<=h@-Z1F3F=K0L}5q4FTiCEwB{xp*Dne6O~r8rjx4=Ze`etmJcV1ZEJ3VMN8@nOo% z9>PMO3xl)eu&gQ|6F^n#NVR*fSqHbvmlDOT?NO#*^J)jCk&5#&bDiA0oPH$;1)4#G z^22u|tjdg{I0X7qbln(juOCy?AN&b=!;@s3^w4%Vksh=<87U$_|fGM@C za&{!`K1{OTAoNWQKdFai4N>-4;m-^({v!7`r5m&vvS40g}kRST$U5yoXhCi2t$7tk2wgkHfmJOF(2xyJt6O4~HbX^H{1F68A&MmIDp8f*Z8Y3obeL82~ zxCfWu$FKNqsJ@-D#< z0%5gdW62|9HeQvkNXVc8oIoyL?s|D){`%|AfkRP67KAS-BViX-Oh#3aC!YY`8kGkk znywe0il2GDT@W*y`8p5aG0~8K^LWNAH6d?Y6<2(ZYxzERrW>*33~cM$(+%LCy$*~i z=WzyHfAFB1qT-xapOyLe`GE=uJu{so(N#k*s(*MHj!y8~R~;MIb7Pr@nW~TGX`h=( zTWg)mKhc#g3PPA`UwxWJsFKFM$NSo;U*joM?PO6T*sen>A$%CGdGP7Au8W1*s%mpjbxcjXqkDSk&KmIr-a4~hy_F3$!R|94CEm+@ek(u^1HdmJN)+OenPf0MK+P}GQy-Lw4qS~h27%wSEY()W8;7JRfX65Re^6V` zgq^VVak9R0)dj06_lxT*@69c-?xo=A*eGLMO1Wbf%8l$*^zP$p%1Sjz7}y9*jL%Fb z67>1Bg1O_=y^AyxOVVbTKk)U{U00EmT#DalX^I_-$&jaxGU)}1a_$};@R7m-B;E^= z<;wC@PA)dE2nYzUtbb4n4jD`E@Wf~g4G%l-3>s1fkOr?O=LO{DM6NMwQGpa+AdphXxbO~WTnQGP! z&|(YqBY)NLzvt*n2l*U)BDA2s*eo?BCI&djQQUT+*A8*=8POfvq%IY%kA0p0OHlJr z)bVN?unz>RZG)gG5H1K>gmt&vQA^|OM%x?X#6x@&sb~+QeY@S6%Az)p7lX*y>Ma(d3Dg zk9S@t$-X5yC4M3#?M5?z0i+$Y zG8*XWlmWvj5gk(RN6r%ov8Pp2C`>`sxth+?WW3Q<{c-Wvb`}+m#INTx1#0XMUHvl2 z@qLFHNBYJ_YLq5Ff&S_F5fR+xn^m`;rsL-t$tI<1=~~I+cY}97M%m0zL##YT>JwcB z*4HhLAIp#xe~v&js;KVa-Y+i9AVo_jSOwj2>$;co>Rh!s!$dT^|P zR@h}#l_rt`{0aD_H<#6F5QvEKqhmuA%56hKibFSxHmYg?u2L6dUOKnCyHEP$ukD%2 zE^k9L4;>;LNE?_8H?b%myeKcf9#7s&efsq2>C@%A^yR!g4UN3e-{pj>y|yo2y40IQ zK7)AGn;9XJ*|2xTbyq*T`K|TVp`Y9DT)Gg^5XxBzM10qkPqmhGR@`bSTxw`=H||r< zgnm_(e0$>gYL~P^r}5)cuM?K8oKo?7#-jip8mMbU0cn_E;JH^fLoFsPR~eJLGpDDg z*LqJbjd`hIU5mVy`FWJw^C$yeC0MF_`}nxpSlLc>jQGo#7+v}%olYmT8rth_ugrE? z)%niprQeVqlfrLL?b>f_uGm4fkSo1=r0r$V9;lN{QsY9Psn#pdxL4AAbPnmY_Yun4 z#GY3l@AP`8t1}kI?0g+{$;e|ZjK%Y9?CI~nL5ptsfV zZ9_q9B0<*Pvi~_Nm^}mOgxEd8aX8G`eRg)_^lW$m*al_>@#%pMGAuAp4uD--FXDA- z_@=2gcMs&Rkn`@3I7XanPu4EfJFTF)D-nL%C^wSfPxh0DDK}w&-0NYfETq zz9m;*?6w`7ISk+~XkW+#%mMc4`7M3iWq$7;cROU{CR%|Mokqf5Z!) zEkFv*P1Q~|dAj9Ke1A=qPjRaPxXRat23&Y9yWd8&Jj@)%`fW{1Suz+t5D2Ue4b$}i zy?PY&JAXopdy7HI+uYJrJp{b|Qq5r^`C)3emlY)|zA!Gu#>P^{(a}l)l$^$0wT(g6 zzT12Upt=6xC08fo)3G4kER~=Ho7=zq@~e(-O;H|sS})+^-lt}5sPtN3_BYevmCnC+ zy^))XpIrOanz)#gwk%;skjl+`TYgbVM5&n!wVP+EFXpM@s*qp?Y}JHFJ%9GBwdq_O z=shiN?8%ZbeR)sGM}<^>CPF8ko*`ch6~D^s?pj82^@ACsv#SN%3zBW|DBn(Dd3PT z5FUvGJ{Vo!+XdUIOS_w9%`Y)7ON$GZb)gO~Z|UznVuQ$Dh)Dqv2A8D4(# zFaziH2CcdBJ~X3gw8!MmUwj~rb}WPu~-@tT(6x<>F!~O_`82 zHyH2J-i|q~CQCHuXPiEM{CLu&T4KMNK;FjbREw$LH-O0N%B(hs z=I%BI-_DOyXG7+GXJE|B#@W>ga~=_wsC~GvPMoxAOJe(xBS%PJ8!#Yc z&bO9+LNsD`hx1iM7KkLE6RU6Mc{CRn9~vTITnJ7DrviiXk`b>=K_?_&k(^t)5m4F( z>$>pR`9#TCInRNsc42s}j~_oOd9QMU0!SMeljD*)B4#=l3XJl#(nXl#xaCJK9IpX< zAkN}Blmd63ELIG5oN;TvF*PcNOHB0KW?mGMHG#5GOJRXycUCc|5$#4M8-N zd)WHmRE!3I81rBoHE0h4&yCUcG?~JNT}SLadwc)IIXl0vpKjsT`lCfz?Hw;)ydZwj zwF2`jSGdEpbp)3Vez{4>pmyEEl^lQq3cwCvfR@$p^ekmRzZ*%7iV^^3*Pl#Q)Vh)UHsld|*IE?$I4eSSb!v^wk@*d-& zPd^IIEpj$a%^T7)6XxgN9G{cDG$$hrgBrpC{{VXCFvf*?8IcinW`IQjoCW9f`4G|N z5faHXJbX8OXK3yS2M5GdK>cXkb~-`efvnyDoa=N(1a=RsBlE?~FqHW+d(a(ow{9yb z&pkG_Kv@9PIr%U|SSc>>DQJCnU{!{I9ox_)!)kK=>jPc|%32g+ac~_3QD+TyAye{Md*eB=Zd{g{FblTYbBV43+do(k8gI+jkKhp(FVcK_+Y%$@0dza^ z%P#{UD4I0k(vbq7fV8Rev+(xzcFE%Gr|gXb0_7!jT&>feHtY|&-qJ*F9r`wH@VzDD zjy!lb;52OLC&Y~y>HZGy%M5$?k(^CuVHnd3g@7?A?r2dZmds>;n!=?#+SaJw!1C!K z@LaGnO*-5@#)PjC%SfYUGhz;+e$!vUZ%}ysX$sOXJjJt-}E9r$}TcPbjgqzD7=_+?iB<*T*C~_2M zI*FFPHS`&fByp9iiG6S1zJ*G7$OXeyJkSeRP2E7X(9MtAq82$dJn}BceNQ7@+oun= zrx9rJK0%naV=XQnRWy_Zt^p9F`Kh~UR#P)Z!bF^tLKvV*QNi06E?+jZ!G$I{)?>q5 z!M-v^G-ba8A>Gpo8vFJ=MtwEb2VGn`>CVcUvB$|Yg4n`6U&0Bm3)(5-kVomd(J7x+ z5=pPDY}Q6L1Vh7#+Aa!1 z7g@+O^nPmYFhOg008DFRuNjjB2-xjDL~L(iZp#9;orn#h26Sa9sx~u!>;Hr2M8PTs z4*x52AX<-8`3pGrY(U#F2R!a?HC$5Qu(IOqdb?YDLfkS3@anD|ReU@Ov&I!$HDo+f z4@Q-zVQE<}8uuIkU8$8Gmk*2NEQHgWLj%8RP2g zPoJ*Kx$K-Y-p*7GG(q_>@5c=smozbO>V7My8u*|URp;+o3|i>glV#@iQx)8p8s*^; zYP;GfR@QFe1lbBZ(2=i&0>iXr*TTsNk6K6B(!O%{I_d3QR|n}p6plvzj))~b^Ul=` z%2e~M@*G=D4n!*zm^WB3Jjwx;vaUGKFqNp>DaODtcaI5+@W?&5YDcIV0b6tEM@FC! z0n0FY=xfuxjVw1AW9yCeyQO(QmD8Hce*QN2>(}3ZNA3xI{r%@dQ5O)YP8!XS3Tyrx z#P1nO`nfLu&4>l38*`9Sl{t~f8Fq>>zheR-Uc041w1F#z(TPXI!a|onQK3V#b zv>;va!MFEne5$6P!w>6s;Sq6bWBD>J3iG`O?tfAD)=^bhdSo)vSg`J3Ha zfVi)x#POLa{^T%IpgeYLKO_Pj1SNULb0{-;RZJUXh0nO;0A~*Fqo4ils?a$09JI|D zG5HIhuV4EJ-Qyx6BNcE9Td>43OW9H zGwHi&=y@da>(5jbz$_NjAy`o+tOf|z)=2mwwc3AS*ZAu4KU zooj@5eiuv({^je|D%dtXzr)+lyOx{*KrL=!66fFAwgq1mhjbvH0^7s4l$3dryxu=I zRKm84P$)dW1vf7pvUi`Z!^z+u|KE)W{{hj>y+G_(yWtVnpa18i+2Qnw09>S{JrDHV zjLDqvOB~Gp_9+D{D)=VB&}+xXr!8Cgt$;vB>jx5$t#52hSC|{_;|57kxf!!eGX1yb zaAp1uHi7fK+|f}sUqblr!EZJ;Hb8b3_%~wS@CD@>AgF<8G2v3U!GC{!x45K4+eZ_; z{Y_7Qp9B6K{PlmhqW{bIbt(%nW#qDCm24Ajr%S0@-D^0`!emyBfJt0*j{jWPzb;uf zg)_U;SXVA<%i+>WGYdC@A}5H4H#w-hE^!lEwoEjw>8yRYlP#isG0!3aOz)l~tTwL8jIIR%{K24c zO5sR+Xy;4Df_i>X>tSBD35=SRY_fl83}LWTLO1-Sn#kcaL2dv!-;0)&ocigt>V;ze zuNi{{nvP`zZVMmX=*;kQJJUegvManNT47q-7s=6{GYUPO;j+c5_+?x48t!+n2_53; zcjN3fIq1j>1f43Py$!2WvMPp>v+MGoU5U7V7Uacl_PNYzwLM(Ha6QoDu9L3LRX;S$ z>+^F{NX=U294|BY9y6Sk%dl8CR6F3X*?^n3Q?f5tp-Q)j&bYRhFiWYSY2meW6QB0m zQ3!1Mou8ji3PyG$8)Sr!)^OQRT^FrS3O3s>>epD$efEY{^XQv7+V`Po0UO+6?t&9J zP;c!6O*K{=6W`2%%-*5ny2#MOb$7k2v%pE(^SStgu_(Px&C0Ydp3a6_`NGU=TyL9e z1tm^kTT)+67$U4j5kv-{8^*-^L$65v|v@Mld+a`nL zuKFvV)J#zcc|}b%0{ZPW4t_=Ad6J5FFCWM!(xG(?)Py>@PQEL5chC69z0}p4^HET8 zGjk428Y;cXq`P)5sr65IFjockZDcUWXRs?Ok&pn_?^|9tq&V+P5$l|d>^fAlS7fBw zQKb8Rr5Lf!S3=d9*Lot?SB6{|OMga*EgBEX$w_)9=f9|o#V+}H`FiDMW;bE$ z#b8;|3FbO>w^%f_t*oQNPGf-xUEjl(Nlw5UCaLa*KzCe{fjH{$sW`_dbg#N_-a{`y zC4Y^IBWA05$Auyh%``F3F4TP_uklsO&!57S$)i!F&KI`x&nezykFAn8I$x>0??}U- z8#Nd&y@fkz5T!cV%9B&9^DUVqWPFR})6btjMLY9Nln5{JIi8p& zErq-Ns%1AFI{99aEuWLLM5B`9LX0G!KAD?CET}fdD$lHecRu5Xa`a#aK%?uZW57I~ z!C4p=eVU5@NL*GZ(|0-I(}buLoSgj20gRLYOy1$l0=mprr1`_cn4z;;?Oma@fe+oJ zbm&O>Ey1 zI)0#-T0v#?;N~QA2&p^+bF0O*w=Bb=t1#epxiF7Y5Wpy8+;;dD2eV*pS>rN1#e$^F zy~bDcF%6}@w-6Oa?b(t;Ui63@qJM+Ic$C}&CQrfGTe!ud@IYujqT53BfM27zY_*}=U9|gQ-;p>ArsXLDHv^}n(I*SCkVQfzm8=^ z8S!iF8ZX-XT~Q{yIfh|Ks>Q@@XhZ`>+976%PH|Vq+EQw3 z&XjqtIdRZ&_*svt3_pp=jFiv0AwH#fnQyS@ux+epF1_qzR$XZxqnGEWa6UVKD6wcVUZhoCKdqnhv_RpT+8qAWUVz)@>Ei58 zZ_lA(hcLG&4w3WQIYy&lwAsPyz7f!v0+OaDgT;Zd7|#)GIe#}*vA`#*vtQ{fv&j2u zyh1Q%NLr0)yDM<--dKJSzn9UGAn(wnMXlFG!S~eMy;Wv~ z0Vmer%`(&&7Tcu2yY;D8p=ayNSe_^>1({QAhvA%OVfE)rQA!inm;v&79FDSsn30Vw zEPVDH9ra|nt_LZ?YJS=|#O$5c4F8y*H2H7%>Hn9o=l=*R{x8Vly669&!qNXfKjK9^ zD3(Hl|AWyxE)Ct0k&qqyi4pGU@mNLhEcCc6bh|8|fEn{BOM-+*03wJ$n~mm&_IuYx z2gQNEn|p3<0QbO;q{i|8NBsJKwve1+a9JXEX0~Ud)P+Ba&t$Kirb%H_r8--AKrjO2 z`mnLtXQ7#TnEenh=pDfXjjh3II%PCbc}i@BVMq1*FXOuTm8so?DtcVQA_EFM(Bmq(r6IAa9{4miVWj3S!Yx*IVLcUUbGJ8($_8J3SHco9+lS#iGw3@2Nvuf zB+vbW2HyPLKfNftFo@$qhw1981d0V}*Y?=c_UO}grL*7)p_1csLS|!)TP^yQ)%sad zAx+Z6Q_EM1KAW=)#;O=aX+nAdV%5A)+g4uZ#mEd;&2s7f-}WK=JcTgGs@jywi>u@^ zDip88n9`F{2AQBr`Hr|FJ`CC!=-eu4{UmiouGJe`y{QU4h#MsU))3N1Do#4Xm8nSD zUEeGwKJ|gVLmY;$*E4AIq?~5FVZWC4^YV~sTZolkcg96I)}eA!fG;f$#e`US^@YCp zmr=3bk`Z@=5L+HKx5fwAX8$;7H za9IrXc`|oErn!XeCMh3wEF`Bak-ZxVi9Oh z;4=du!huVNjSB_yLS^&bBpmMKk!I{+Au=Ynp1g{aVehSb;54Q{THBfRIkLicag$gd z$t@hdX#ow;!SCc+W}#M0pOO5$ZpEkhg~OqPs*_)REWdsJOLM1sw*a^^r&8bbNrzg0 ze-d2c19vc7-=}YOaT&?tUZ>`Mw)9K%l*@_}!gCSrKiQE#Dqxa?$!pK8fs zu}$UmjgVD7xSUR9^>-_xSCLLidqaVYzM zXeL)Zg+wUG5ZE~nH?s>4hC1cWPEY7UQVZns`#X3g4A;{SV>q3?$RwlFOWdzWMR4hj z{P&i-GWe0~a%9@j)}~8>fMfE2iJxzv{BeToP?k0lK1ZR-X>IMnYNn;+yyEs#l;*WDcv5s3o0r9wN# zRTs7igY_mFU@SEDmhvEh+NV#R3kg&sn9f09Mf&432g;~&y|OFwgS9c81%7_+9LW0~ z6TYgu3e-UG7xi401h1I4;sCo#B-SO`#}5GuAAFr`;sNR3N}%Hu&Gy~Q~E>Bbsw zZRIRAvdXz@sNA7g)c6?J4S2M058ZCRK=cC&eN^#yuD;Ul53jfi6Pi8%*@D|~$mSUfwRPzlW>O)E<~ma;|YW`bTU@B~p*Tyekr^0^cIW|nV8(fS1FJa>Fx zWd6+UL@5Mr6~e%PxsL&{ijeq+XH&aU$5%YTR9JH`(Ut3|5TqG*ebK_!zMVKr#7HH! zyrCp8-7N$Ke<5943opnsOd`d;bRsj(QOk*tcRi0c zIv)&if9(M`*Z_ZnPGAd~o*pVS=wLv_9hHO!$N`h9y5r&^(}WHaB^jAK+ zJov5nKLJKes_K*W4}KBU15-6`jAoKjM8jzdq|>@`BpW`4qypt_=DSse!AwIjxPRr@ z@n%|oK`pfiE~il{+;R=KQq0cjg3%$fbFih3E&Xp5^e(vOm=KQ)ExukH4jdJEW>J%& zwmKnRnHG2g9>63rS{@QS+Epy^M7mZxFE=ZcGPqscy5NJp`!D}ed~QDP1RWeZk>?qn_Uz%8rIWD!SF;&^s_yNCW=xB`|Tt4VE%;#ZUc z&BR$wxFnsJH=H^E|9S`4)xIWU%Ivy5C2ljK5>GNzIp_VSZ@;7AT3U~H=`bO`$7U9C zH3>(16#Efr%{K?K1DO;!(1|ct&-$w_V*DXjgFU|)q&L642apvfBQ`U?>BX>%CdQw!2L4^8^iDer=kAY>uy67 z5#f`f_`U*~6HZDm%1tgi?B}RnWOJQ00>}cM@Hu(Ry~OC|5;oYmIWFsn!YPUZvx}aP zZb`4%YVdVOAi22kaV8X&v;`XUhdf-YU@xFAsp)_YPxlkw%(CJ&0dT!(*q%t~RcOod z=}E-ptSI<9%VyU@xqs@|rRQ4J_l6Ef!uTN0?e8>K!5O#*`R~SY+nAaGIUsiTWsV{I zB_Hc+Ek2IPGHXuh$p^POE$g?<`l!W!X0n$Q50273n8!+WvYrx9rZOm@$z0rYy^!dd zGK0&IpBj#tr^)hkxw$g97XIy6KK-4i5D3oi37nF$B^O;e%M%fV479(C)O$0o^n&`R z{4xcjcFqa3!`OGbH-t&V_#F_YN3>nRrMz)D&SR7LB$L+ni0mirbq8o48K*j=iN^NX@Pv>BZXl7zpaon!2wM4R z?!JBAtP&9DPzYCw`i}{T-;hvZxfsd96 zl$dr>rfcd7hiINO_+WPxbRs_jFQlr79(slf0=bwDY|ATLc>N%~e4+ zR~@b=&c-xY1pP>flO}(UITS@E4C~#3I5ZXw zXew83d`~9XI57I98J_bjXWW&y!bb6??e^C!lG-G27SYl-f*aMIE5ZVYZjBgq|&6S%%NNIA<-5dMb(8Xs_n-xdb3BL~N0p=*70>j9Ziz0S@thhssbAiI5cxV9Pezp1!11&y9kEEqoQ$&Kt1Rhou4i70swGSFc(nI{|VC{{dVt z;J(qQxU6t|W@4HsT89J)7TMq-NM4+Y4^!JwlS7-#!_xRFTZiaJ{b$t^Zu?=q{s&+k zOpc8yxm{fWpLuC8>2aZDRqZppL_Ov1FjM8Nxm=Jw8(^;vZiW1eoj?dAqWn_0P(GiN z%DbjwVY5L+^GO`;+19r< zWKE+&&9XH9o1N}~s8kD?+e#2iO43YUPD)CGW7tX6DIMT&Q~2u@aBqkmw_3K7Ea?HB zZ`9NW4Oe`8y88O$tO^y6D+sS~)4c&}PZWl=S$n@bcc(>0&Rr9}l^hV8FOldkIX|_2 znn<*H#xAXbJ{|77W60c4rNZv_IqPDsD&$3jyO)4VNJ2}k67D}r61bj3)sCE;on5!Y z1{+S<1fF0b%8ZBlv&|c1OcRy&(C_^ME|O&qv#P++2kvn5ltwB24Z^u0s7na^K8L6Z zYUA{P&puFn8_lQFWNQ@UM^LK}n{OLsNqB4l@AE15QHO<&OE&Y%s<(dx(ru*hE|-IJb=LZz(x@QKSeqDb`ve4&75gAC3JbwrisNFK)zO&v@!A9=0O%;+@v-2A>n zE#jzW0Z@g(NH4K(VJ}VOuP!Se6xnG`@}U)I&|li`eYgd0hZ0WizJ$=6!x?M2#;s}; zOX*K3Z}O*RDr^jH8g%fweju)~{Y;-dUT^kcqP5&__kwKc8nBUWkOrjpB{;u5-f12l zK}^T@yvO5WbNC2ZrJ^GU6**qqG{e>W(gXC$tcA>db9y&+nv++h#mjaGd0L$2A5jI8 zU)Di9hVzjuyt|37Rs{Ms}``QtezlMB?4Jz=~+k3Lq; zxQFH6UciCh&d;g)Uq^o7kS|)A6|5N*xXAwlD=rf2&I1cpujbjVI$uA>Nbxz$k;?0% zEW-HeBkrR}Q8kg1%56esN{dI5=I~b51!oMCFmTP zfeB18&Pe|lcv^9*M4I!(b?t8O&uLIU#K*^n3roelrn#GR#K6G`t;JY3NwuIa3qJ(> za{apSHiUquUzLT|nD5N{#xn76$0id#PCdx*{2y%%z}vc@;x(&oj|r$S1etcG&6Faf z^PJNEor&kT@DYZc+x6hsnKbkW>Fd2O6~p&EbBp_T9(*9z7d?(C9!zX@Ht(Oipq@U> zW&V=G^6uXOzC~ezfa&cxI!sQvaNN0@M+=>s>P!4K(n3|nkMU=K^vSm#K6)e)p}Fri zKfXP3dNTfOI+i1W^+7}Af1dZ^`}I}DA4||JkAu2Ot4?Va-Bdrt58(Z07JNjmd-Yrd61IlC9S+W(=dkQeYrJqqRHBU$C%g;e#zBzmGu9ccP~d;hZ}KD9j8 z?cATd=k{|s^y>SsMFdc>xz2RB$AezXZvAqz0KW>lE?yS?6jF!FY()mcI+r}5k z9)GvCHl`Rcf1X{_(#V_q55b3M!AEe49(yol`H2tt_Rhh5Hm+(f<07I-A=kvtZ+}SM z=YkV#phiCnP(&D)$xL+BYRhJ0PG~2d6Fz6{RI6_aQ~K5oR5&MvdPLF;{&eSULLw6@ z1mh`zghGRQsquuDt8;lMt0aLJ3V!ss=2xHEkmAp3tWKMc-KfH!|DlCPj`jVM5T6E^ zNPmW;5O*TS=)_nbsMcV&H7+hIYh&b1_I$51>q&9UGkZy`W40(&`@nfudx8I~%bsd# zmA=Ovq?x9^K2Wu^uj3o=1>X8HNpg;7o~otu{CPn_$dpR3^_)&FLEFRlz8J2$>Wu%; zey>0b9t4Cbhzwgj`UWfJ3tyMYwBo*f+tUIXO;XnUZ%57gCm$=y%gc8t)Bc%J@V}vC zObx;d=Ofm8sIDIzE`MJk0B7Eg7}A`MO+s#`?q;DNl!Cm%=UzXzpWa8SKi~Ge43dIM z3p$GNEK;9_m4r!6he)!E;VjaC4+}C-e_bn!#wKDwc*NHaf$@RfQS46Fa5?UUF&mG_ zfGijFDuuzmPo|2B=v_B0RN< z^R;@!J2bQ`{2#_b|LOZbqx#zk5JdsnDs5exO%eLayF2kIIQV>_C-mI0h2&-e8v|l1 z+CxaK{~7GwIEb$N&(wx28NiRYzpwr{CTxZ}32}vhQ!^)5B<&CE94+?#xtaD^|c7->jXnBuapu7e249ccdYxLW9l=WIGFl{cm%(eF(S=lJIsT<7J6Sp9&#*sfEV z+&J49n;~L4_&9s$d5&IKRz_+Zk$Z78@yKe?Dm_^U5%5AupdVqkU1R$A=8cG1bAPqM zP*=Os`}eOv#0HXRG@;)jtpfnoS~Q^z$!1Qk6SbHSr1*-q9)1mhQ`5T zlyexb*`7$v)c^t!2-LanhK~i#T_8gPOD#&=n4j7`H@pEf5|H&g4Si2z_#g@>>a*(XWf$7-DU)5$iXj5J|Zng=4dz ze=9`^W}Wj{v?4Wf-LPN36a2oODz(lf~fDw9gKnR9;!x&xBq!EFv zG$S~*akW{)A=CLrG`gGF!x+`zU=@p`(I3g@u2m>TX%VY@U;7z(nNIcHK-m$4RmAW= z=VL!@j&~5jWez!p*XK!cbAo+q1ix$ODZDpT>F7tT#E@Z!eSP>8+kDD&ozYodwVMTH8lQn`!^;bxR~?z z1?S^G{iU?)^>%(ex_>o+Vc(nm>}*r`(_-D(f%e)#bT4-+R!@Z{=5XFj)QVj}=Pn3P ziyyaLk5xS;VhVfSB8Kt!*+odoC$P`w<;tEO;<`W@1MpZ44>TF4?I~cY^Hpw_cR^|r zSF_D%?JQbq*Sj#%EG2WTv7CO5mPlO7FJHW^PYb9nuG;z@*J&&}_rqKZ9Xc2QCqKq` z?sNrQ@?QcDqmZv&Ryma>775tR0&>8jeuyvzlkpjG zo=TbU<}3B;Bu^yidY7?5wC3E#Mxv$eOX$vcOWI>_J~G#;m)LOwI00!S))oW|+LJn( zv0>vC4(Ar0a)l9;PR#Aj?;1qpj(938q~pyOLPr}EM%Ke|<^i<%-e0^2k`Kvi)lO(2 zprb-X$Eq*Ye|$L>jcA$3xCPRmiuToCu{m`cG?JL9y*R?*GMCBdO;t#Avssx8aNKV| zB6|5U3N5}ysydDns3;dd!$crxmv(o++;_*qxbZpc87G#0ipWr?KW9-8@Qq|z$aA=l z6_~gqDHYe7G7U#LbEI7_l8o&{lUf)i~4b>I#+#RE(T zxc|ag$#9aZc4fdE0yy?pfB#gMFQO(hMO_9n&?eNm6i-LTx6f)EC1a&;~ zeJMg-whj4;B4shQFV$%6rsms%Wa8#e=0>cI%66ZR`_Blzeod4nS|eQ(Nax0VhHYw` zee}+eVA=WNYEdqDupS`+)_@K|K%3eUZNAImu-3Zb7V2-iHCJo5N&fn7DI*$bouQ4z z`)PZUK6k?XH<7I?o)j23%|z+^TAA_v@j&C_A(r77xP!x-+|xy!IT;pjDD zuG1wL-B0T^VmzF;d@_ST1Y%qrN&D1#h@|@|+rmJ;F zdCWFIU;49kt(&XMa@-b6VT8FFB_W*lTMW9LHb6?sQp>puj1Mg1+n@yQ&feY&88ReD zt?>wxk^T6>{rg9b7Bdo};VvMF*In#k5RExIQpQ+gXQDR}*wY=)J!QZP3b6U(a86sq zLA--5pbJ?8(ahQpILl3+zj_(G{gnF7@9wb@e{ALmPVs zc$HWK_`w7HftWrKpyIs56|~Ro2HVRP7M;1(3!6b|778F%pw&m)m=LKcH(eQ8W(4QJ zz5hMD-u|^RT|$?q!m|A9YU+YcgU2ffTYE4+%NiBqC?j1_NoiKS36--3rFWBU-Z}a- zu!as~`v3*vI`*DW!Q^89Ibhb#deJ?eaoatc@e+Tj28}o4cEZFxd)36Uo1v;$e1Mfb zywf@~_{;gk9mRgQbb1zYlLM6{8r+9Zh{ zj$fP%Xz}BzRUa~)wd8B`@^r^BBY8L<^R4{Kik($RwsgOLweBk?o24G>t3DKUJoV|V zbs;FRSDQVYoSTD`h<*WdPxl%fvQyMu&*)uSO+aSHFJfa44hVO?%#O~1(#VX9lUy_s zm-A9txYkQu$T?&0v}M5*b_S83hJ%=HN$k_OnVpAvae^?anQl2QKp zi)(RQwlU?K96LY&QJt!*1Y!MsHq_kC?ydE%NKtH#xcpuL?_QDiP1PK)^yiW5cGk=T z?ySlCkTZO2PeTEo4;kF1crf29(y4iA+l9RC+?YQ!{)CaTYyRHzBZe_}*-PP38EHxSk95Noh_3)XC zx5-#>w13a|c#JQ>llH_0h(0s{L5q&ze9-(Im~WR05v1D)o*tuy{Nwg3d=p3UD`V%3 z@RLy{9s1!oq0Wn@sgvcd<9Ol_0&s?|2p9EZ3JCO-{8Di)DTLGen9U5sFh2h`S=-75vH zq-?{U-foN=6U`~N*{zp;$5+fO&83Oy^j_V4!c7@WtHxm@TOgVfL4&gULay9IX*XZa zlNXkcm{>gxj`e}}_7FT4vn2H?jt(PMAXhP&4SMV^(_x=g04DeVi}o$8zP=SZ=e}Sq zu3E7T4y-E@9So$IK}AIcg!jD#Q0w=?itwUnc%=m~O93(Q@#@Yhd6>ht8iTG?V^1De zWPvI@PEghUgWm;I3Wk%Z>|lZX6JJF~MPY{?qAZ=u$ZWeYsf%*Faeiy0Yach#}z39Lr4?9(EH)C=TGXgclYPgye^g2Vm&{7IxW>!01L6~ zN!G*eKKuMbJt69Y2ND)rW$V9Aw;eE#@RxWhDq%c@VV;%1XGH|8Gf1_~`$ofZca9XC zQa`7mfoJtRpS^e%jy53|my#hot{m_M0pHu1P85P{T>x0zzlZXIiRrP12I0mA!+Vf+ zu6dVNdrj4ECQ(@$4)UNH5Ndb#+wY=T5LnUa)>w_Sd`PNJk9Zw@?*_z51FniWg@pqVX3I+cIM5U zos@n0FTq(7w~I-}=(_e|IQQbqAs%aq@i9S)DS!|D;W@*voE9u{^R#7L)(#R~n4RuW zMbK%^TwQLAR$+QZ5dNNNJCCW!UACz?n5Db5Ci1yB)G+pN6?U>-y~ zHo+DsyaKR$N+~dvF6Q|_!b$_n2cb3g+w?14N$+AAKANhqSb!+_r*xMs3*{;m+aEw7 z*<8i?&r{Av#m2|#DTe?z;IKPUYoKhH??S(Uh2|slDt7C6AQ*tn2fL+Vv8{d&k|G$i zn8$~By4h&9HnOkT_?_76c&foYQ5DLrRI-OP?OegOzuZF%I0*>F6sUJ`3^<}U{K3Y> zMfod7h}8@A^!XbUgnI0EfS||fw2|XEN>gO8c(-^RJ}9~`9H&sL6#LMpcrog2IBd-{ zm>)SvByjWqnqu6IEf_1bdjj@7<6vLs?Ui43q*0&=cp?0m9#7O2uGx*K@pKLNAM*A=k$?({;J) zwn5d%*2h%cjq)#lLcc07=K~n2E*;I8QDHI#1r=z=O5K=wU0^g|8Gdh6?US8HCSuVo zf_EFE)!#s*Jfqg2!eq7UL?ODht98RIjbK)Df}^c*LHzA;6zmgHaH#Y|(5QT}HlI{T zxrK3n-*KA26zN&%g@Px7zpd;bZ_@@O1HGOoQ9W{U2xVneu~84lj~~rSC3g6y`wb!q zTvb48?3t1hnuIDuE(G#DRAE`6FULV2Q!9Fa+-^Gy9he{`gYcIJ`Cg_y@y5I71F51L z=L1bmAI0&8-`A@u@vQScBvLxPi+$4a5clNF;^a5MG^MN$y>=USq)Y60WLX(AfC(Zz zZa31KW1V7x@$V;=htQUvtfEcMAV0|4rEUo?zjjxr!{2+04b=7efQt?SW}qeD2YF?6 z^6(ixBhIjppx`T)1%qn)b-t$Cr>I?2&W_SVuvkc#o#kwQTElnTT=g3Gu?VyQc2+vQ z5-E2NmFo2H&GHv%TEFqQI`Q%G-Me2EB&lXTIjL2l0mP=EZ26_BxfV#KY}Rw<A!r{{YVvId=XS>-z#ev6lRXLX zDwY-&-vjMwmU>_bAQ~S}%lv%<0ei82hS3u`2A%dsp#G)MTyjJ+9?oy8u@mPHi4-!81AxCF1~C@1;O0xv{y)ZhIyP3McCh3Ictf^D+MKON`b5W>i9U zFCZ4T0$c%-zcc6!T0D44{rcTI6uja5=VB4v05M99b%n~g&xqW950c+as&-rXmc^L0sut8wo3=8MLB3t9HoB9bpUD36D%-Jb*ggH}p6LlOu#5n>753xH zjMiG%s3A}>=I&&z_q2+~W&zb9!1?Kd0c&0q6A+L)^2e|#Z1Cq=cmwv?C;fAOfB&CQ zXdo#s?87K6P&?`oGAF5y^>;_HNA{kf>x|8fYQQ98dHl}m58Gi&v1CCo&vY6#$Gyj2 zkQ)S=z3vm1UN$Eu`uO@LTc&v3A0{0x7~0R`2T7sZkq3|S+LrPrvXs-TKG}62X7t3k zMJ>4P!(qcoS)}X|boF3$VE@Dt`_{*gkExVDo5qgB#^2uYY(+fTp#srM}Yjt2G` z%dHDXGB-@;gz?c)Y}cTpwN;_eAZ6+7QYs)kb;ojIf@F?B>yqR4b08}bllkzV3Et{J#(c*B`Rff&1CVh^S22=P(##C$&I{a{L=7ysgchV+ znr<1o1hQSC0hnqwgZ2TO5nkSC1&T$!oR$es9%)2SD@C!|lZm;XhWo0os=40O9d+z;lsk<4Fs#t;7yXyLIR>y%DP z^M;N-R@_~LhWi0HQDi^Dk}bcR?AR5AD%0G^n^*-aY8Lb)gaGSWGw1-?*2EXU>PzOh z4k{8P@j|tR(@ur&7iswxTp1J3o!aVrQmC* zuVb_Pkq8o&(ZX!MfV_QuRoc!aZD(B}gzQcWkS{ZoKNYmw@bWmpz|(u8dOllfMAz~Y zvtKF$d~_d$i8~;w>WV4taw!LQ#P8v8T>i$Q2uKQw{mS5#oUucR)*>#@N1N}t+HX&% zz&z-7j|^V}*$(jYgpSpx+Q*M3Sds4D)zsA$2z*v#PcRM)xTPW+VaiKqQerX;Es=ZM zA}T4{fgq*luZo%1G0d00R=hEL-wFv06qOJXy8ZIO6mr*|o>V$m8%R{CFjv`F2ekEj zBn6!@UAz^H8knmDfgKLqK)$Lf7P#Qz&cV*ndi3?<$Bs_c#;~vRaAYxLeENH&1fBSp zFly>3dSU4t2W(D&N`xou@KFce`d;fQg~xZ5Q9No-TUF>5xPi>Wr0Rfl?*Uplo0{x~ zvlX`5?^+-sjpVDb)+X?pvwV5*gegY49YcRPP7n*tY^qlGBs>)u1b!rbj&+#{r&3t? zRgVu+u&09jxi(wl@#n|e^BZxY@-(s5M+cV8GgzKHtTnLU;D^OJ9X~G2^@|Lfjq=qR zprtq^A&;F^LDuAC@%0E7ZntF3D_H6Sz~)0kF(ngu(f}ZZo9MjCsG?cz^@`_(9{`3A z>|A&~r>>wZz11Sa<@(^;32QqR+)_~qm59AW#tcSa(q%fMHXhE5_#W`6xjywB4%SkS zB-^1k1u(aSfwbfdy8R9;xoRT4TVH3gLNK6E_JF;;l|B+aetyiJfHaE2xvm&e_^I@n z=N>b)^Um*&vwSOETwdgo2|GKQFC{F~=Wlmi`ok}r0J1(%SR*w;X%PsSFM|Q=(VVHj z?BGZT`u<;yBi<=(L3j~QzyHAe_j<4I)(*e}2UTLE;Z_a5{j7 z6%`?>eAPsoZcbUL$Yjh4XlX3=Gv9rcUc7hzevJWLdTx zu#nCaCdO8n^?@X5t${d(Cy3529jhZ-({;v2w9l+B4Mop;0GPwtJUhg5aB?cNI>lJ( zjqSPE#wylcHVd6VJF|LOWyM}%c35DdTHy<+vqEBb&7*6@U8^03)fv>+>*BZb9k62@Nt7T}*Y;!{>_c`8YuPerT zJAeHEs2VQqKC4#6WvxgxSr8;}*v5dH{g#zQUo4;RgM$@JO>nn_L`SDRKnB3b>5tDC z!p&LnKG#6XsV&;+RTf6qF(W|uKwA^(Xss_*+RoWA9dx65uN_zSyA@1OR=*Ux)D?O- zk}^E5L27+dw>t%CTu}G#rIU8IAD(U+V(3U`Mxm?Av@mF#ZeaQl&V5aT2rWbd<&5YF? zCnCp(W^{k}kxBtqu>ipysK^V8%hSOT%3?CvUA2$pu(xm@Btk7avFOEPHSfz6z=Os& zygsv8E$aHYEyFc}h3zG|HykgDbvqn~A}R$&0;_rmUco5^HpG)U9>;yfCu z7Z`l?Kz;I)L@ohg;d|Cg(3AwpegYy>3Ycx3agjw|&}0o3pTP=0uw$8@kd}ISA^?LL z%ex-wtYjphvlj-*{kZP60Bn{aNdiLABqJ_drKaM5-@_!@y;m{w+*=&hiEq9&+PcOLjubdKiTsxih z(p`|z-9vJ#rbDezsO0m>0Q8A2SqcGq!>dTTL@B(8vuHpW_XCF}>E=sxI=VlL$gmwu zJOV-hyVC}nes4mW#q0w5L;Y@p-ca&Zd*-2>^&TLZWXja&6(^j37|gHZ1{%`0f`VGc z#$COBkwsQRxNz(81tRbJY*~?w&5e~~C3A`RlPL=!5x}5N+^$r4F1|kklbm7RaO=W$ zkrJJdM{CL}z$+^O2qmkNWk`vort!aB>a34x!LjmKxK=|22xtlgYV&|tsA0BG5}s%i zLhu_BdJZ!p#)D>Bb8r=`;3oQ~W=^koD9X(yBtYa`l|9{Oe;h`T8c+MN!sOK=km10& za|aQzZ@iGA1DjtYwf5XXmdB~vU~4N!Q&l95ihSrMZ|`;o zsM#)4BFKE^euW53)8IKp`rAA}joSW=8)>XqcfKw0>Hfj046u8AM#iGuJDT|NA6FVo zf~(I!OPs1DU=M<(9GBxOsvLHoBcTz120j5PkJNaWdZ9g`d9F&Y6POrihJ~c-dG&jv zg#a}T=y#JzCQL5$Ch{tsmKPT<4Fsww)tDgw5X=^{3X=&jmjNlB1r-&eW|LNK^tvJC zwNDX<=zIa40ac0~u=4C?%A!&k8(&mMCx+bm{%%$JU}HKwF#d8UYX%z#?wG1RRKYOu zGxy|@Jk$x{I`9gYj0%WH3msg>ch(AiT$LgrHsaPGbbDdN&45)GMH4*}-^ z(y;=CMiK^)dX{tgNPu(0&oo&Pr{jYZu?hkQ+3e0wU;Mxp{aQ7(Z?>y-u(Qs0j;5CR zzKEf(ug}JJKX~@~te$TTTmm$U?bc)}0mKs9HD;ZiU$ZE9dx)odi=;rw;$*Yk*+_SH zpxV?K0G!)IKf4jyksr59w)k$Huv`Q~FlAccib0TQ_tjPG*Zac(IP7uXzklGOQqXVB zttvuKV&3-;`UzM2-F3Ud0s3B0|LD;1eDjAOx>5O0>8lCXelH+rN{>`LDqB0TTB&H} za>c#%IwztBF3$p>bju?zFHciVi;JpSCf~#m*Zt%XiJ+?h2zk4IpI#$Fj?#JeWLI0c z%rNppd|GcK(9Ml8W)!bvw17Ei1$F47UR~w9Xo-48;)S*Z=M@g}y1+3CsH>Vb7~cX7 zcMC|=7&2)m9XPvDWi6X69ZEA%zTZ<>P*~2gB`?$88ztS;6Ao~s&caH;OJJb7La6_} z@8%k?g^k0a#RN>b*jMSjQKrRt7pMEyr+XS8jWkZg>0n-SicruFJ-q`wd3TE4L@Be{ zjJb|G0umAu&^Vzse%{rfhn*8z*dG4?YPEV??jH;1bJ=nw96h{7eCt6Izzlj9Y#Eo~ z401>;tNCJgbcqY2W(e_9p!kG5<970Yqk)Equ{{#wJ{T4lUo(&^UzdRouRb+Htk}Wrz5n*SQyz53EMTg`nQIsrDSW%%r9(KPwz5-(3OQ~At6!3^2 z3E@3pbAZf9f!aS67$4SIOtj7hJ30v3T^{240;9v7*eD}06+m76yB*S{NHKAXcYcxdpE|zQd%f|M}efO9$gZK}p1r5|op-1u2Cp{yT z5p?H#N0-U98_&qvhD|4;Ul~%akK_t(zE?NgN{~A>?7RJ7f%WqD=0c^-D#J7nDx358 z-iO5Y^(}BDSp&by)&OR1-EXO=z{;P>pOJ4eRm#)!U0@f3HdnnW?uYpk(XW(>*K33Sf#E3iMJ(E5@O|Ao- z0;NhgK_vjquHntWOm$}9CPz9|YwH)X9qB;pN%$N_N-lXRG7*G{W*|0h1**A>ml|m@ z{a@_8WmJ{x*Y=H~qFWFIr9)IeKv23%6cj;_?(XhRQBWySIz_s>Te`bby1N&!-ns6* z_x-%jr}x7%#yy^QjQ^KCh%IZa>pHJ_&UqZaInR^3kdeh3bYGo~VKFfNbo3im>+Z$x zSFFZT0>#3AU$;kYz!=j5EdOP)))Jfrk(|}LjTC6KV(p0eY+|3x0tdU?D^Hhot{~K%h!AA19eh=k;*`0MR zlh6AtP2}BLAGQ#;tyEY%LG5l(7)j)~;1G)i&rv}R<=*zb9;lh!3}ruOEPk}v=74C{ zvpk~h$`Fs7U6gkU&G7uCN1gey1YzLJ!fbG#SssTSkpCgMtsZoX{t0FW>MK0E)y#MJJd z+{VmRQx#(}Zj@?R2QdS+aM4Ja^%o|%SX#ex?RVVykw0IPV3)acUa%mFh9ZF`%dM@z zwgrhV?iag3*Xi)eoHM*_24#=m6Q5?TqTK#_E zymP}~lGX3C!6z_pCsBnTJ^ncy<~6?e@WI};k}84a*y`z)8)fh=0vx6@F;PU+jk*V zg^l)2pTk?C;)!h0?P~~IS{1J_vvI4VNhe{R`I^X=aQCr10{BhCW;u^H1q9*S0LI_Q zDo5JT%%K!((KR&iiTfqz^Uw8M3&Oz&OW{a)a#?i?A_!D7Q&4ly4CQ`2rW+2fMt0YvO)Qctv=win7Q*)+=u{5CQU~*h*!Fu z5ll^H-BA||_4QVa?wbOhP+h~McwX6;`RYncoMJ#Jv0eTRm48W#;XCy=8; zkf@Xq7lEmMjf6x6&D6{kjgZOoxq&1_s%rU1Z#y^~z{A4Cq@-G%?}8K11t1t`6>_pD zEdWN4hF7Ka-#s0S?wV;agsk{*onvLHR3QF%y(3uIWpy|iZ>r*$xniB_KkUN)B)fnH zMgM8qD-y9#t3-=hua~cy7=ylk)!rzxfRywvxr~t|*fvSV!TL|_duD&(e5AFM#f65d z-yjXfti=sLyxq}I$od78gX}>@bJaHf<0$X@^a?0iOizvP#C0|tVm?~44B6q@K+r0S zKI$Y+Re+B`&cbZLAA!ThX_uFbtkx)k#D*eRj(LnnvLyjI8q3>__sfNBCdCZ#4uyBC z&4`emUSL14Hx7WXz~ug07gE_}AWw?SM!euCz&~qSO|Yi?Q*>5=065y@scUQ-AY?Yx z{XqK##0gQDU;-Eu8IT>A2(Quf&M;)fcgS7l;h#ZtekXk=b0otr+Lu7sH)syT0W=hn z(BO-0G?Tyr+l(&Qog3rTzN;(-DBlTxp8}xFi2QjgyW?MwwSiZ2Q!ts14Jy$@xoG<= z^};)zv$H(99bsL%ro+R-vIl8`m*>|xZH`EBT!P0^-F4u4&(BM8;}8;FYkWu24{#*N ztS@_}<99(&k6^aD0Tn!(_@Bcp)bsY>=-r2Ai71(rDxOVlTj~c7dh_C;2N!kvwpbXW zG1Kv?>k1BVwj&XX&N@G+rDAd9DD>seAM;%n=(A`p{DpxAgu&x8q9~m$ zLKp!o9_x&S%9cYj29p4zS1ohA^_uf~Sh>6Yfx3Y3zEz!~DpZ8uM(ClP=)2WnXIgVu1$9s%?04E8iR~x1Acw&*1O68iz>J~>+ z3f~~+Fs?h@C&0W=(bU9&6S^LT7qXPc#5z4v4`5WP@JB9o_$DSMYDH#eGYDf~|1u1A zKyl&L$zrNyIME}}z+iiFfG1XU%oxecHM=Aq2SXzw{97iSuJ4S5X0n?1lfp*_q(vM? z>$f@I*uO|mKk5PuxADHl)~WewOyHL-gNR`8UG6^<17!DHuGq=of7k9eYMBBnZ)J=opg)oIR!X6FW&o}Y>WB~hW!p{L{ z8Z9^B0FeMlG+bihuerI{_Vz?+Dyb!ZPU4yDa;Q=T7yvqiv$N2`Y6Y&Oh(~+D;sgSa zsWEuc6#>c$YMW6S50$4Vop0_^>7klQISK_4jC$su9K)!=gbB_1RFLJJnd37hk8teA zF~(Fiu=Vw&zjYY*==X7}ECuN}bcN+!AURL>$*mue{C2n|@|VBQ0;0!?U4r`9JxKYQ z+vQ9M`K&pIhx2LqL5hKb=)?Vv@H`8oJBUWN>W*)FEWliI$esNo2U4Fy`jB}0=aP6l zlOLu%5oeJHff~2nPeA-Ybe#3ry9d0m{&>ijr+A5-rl z>b^fQ(??YGH(iZ`CK5p2q4Mdky$P?W!QABr=>5RluJl0im&e!?=r2C2IyScgITGa< zKr@6Ncz?v3I{hE5935qUf|v}2&#MoFH(Z$_z7tv z`u<`Y(Hhc&@}~1=ie;aHvCMPd{0mikyYVApc6Mksrt`0Za_)ZOPF{1kb~T>>Yao@w ztyBoO1W-oi+C%A4HW}b%0K2ECga?w2mg2$x4=Y0p4(Ca1SvfJ{Y4#_62NnA2*2pK| zB9sdI%TVD1B_Q>tlYj%)3Jxo+Y~F+Fvm=fy0m~g6c%?j>34K#jvlDZnjeAc+>6{9T zo1_zK4P@d#k4%>DT02JceN8%3{R~h6egigX5gC5V7eP z1bn9hNdyqoa*SE z@62n{Im;#5O&oz@7H@D9(nh_djdIuN8OJlKaA_*Q98)MA5ZI)wjcZHFaouEa9$lqN z55KC!S#KP(JC4Vqz~dBx1$@;Lm`xJNYsCxlM^pAhDHh`S2*)wW&UL5#C9Ulo zv%%e*E>ACwt4oMJSXES<$IWhR#IHvk9s>@odS?zevB!2*CV!Or$b9{YfTN^~jEzA{ zn_X(pU|R>=(V%PGZlMQB4Tc;w=gFPGz;{0Eq_L#H`7Wrp^2OM2%j9WZ345kd|AR&n zq~}NX>B@*p*c?z1B!E7DTM~o~b~8ocQ)|7`ETo{P?jniiEP=4$?0Ul=Zx_c*DavLb z6;=1lj5T1R0wIs;^yJllm1zry+D!YeL(Ps2(CpfUZI1ixt6ID|GJS28&(A|B>6-!N-puad53EfOab@Z!)jd-*kqfD>^Hi z{H{o)OTPk*K=RA+G}~BUadD`Yb1by8>|MC-qYKJqIuZq@>$V-2+pglu(J3%zA)P0$ zO}peeerIG?IM}HhA1B5>{_7s&IX*xQph&BU1Q7>T$Es~Sn>3M~2Tmqni`8iC;jUM9 zBMO)X{GiZaS)P=XG;LnZ54dgv=>PEz0oW7+O-bO1H24%}LZcoF$eTds=nl;|d`G+YOdXSB#QM#fe3#%H1^!gW~ z2T+7s9yHwl&!NX?KXj{sJTb(B8?@cIq=4RZfNj9V6G!1vrpd@e4~6gEiCV8vH1D z1_)cL+}W^3@xMK+LRzM&e_4Rm^||6Q^o3WqFfkj)xKjVvd7WDjt9^cc5%(a*=5n8W zWKWM)JuB_cA7g=zZw=%rn0bX%O~R$Wm;I)k3*lB|4;@8$4*dPB)358NkB7>aT- zxXbQr0eBte<{1T#DQg)?Y4;l9j8KB0OlMsLqKio1!&fUKr35tZPVn&Yn-^CYXNPbbD3j)`_CU>}U|$md)1KOCMsPSt=j;huSUrKH|v8HrCR znsJqzcfLCpHxOFQyE>Fe0!Oc>7-Lz;XQ>LG9jr>!Y;R4u0C-LX%4(WgDL-u8EL>CC z8~oacB>)!zJ!%bpbkr>jCR`NzmssfzRRLoov6K_(OFknL=zc%hxqEg*@*1Qs;tR{U zNJwY6?T_Xgu!*gpa{^X^A4q&Q>qnc&LWi06a-huBk#s}ZEbBG35mXF)CxU~5eWaxlDg8L&d29={#sJ9XP`KY;}!=O?$Yrg(SP1wN0a_OO~Gq}|sMR4}^m?BpOR+Ch>^JPGH; zP0EirFFpsgU@)OcmlouQzhfI z&CNxY-;tHM?q=gyM$kLl^OZEbA_d%KBmQ}2A-82w$4EOg2!svZ#?`6DbW2X`M)@9i5xmwdT7 zs5-#Hp+HpBE~z}eHq+S|bLv$n*R2PqrCBHa5zxXcHc`npebwDdv37o{z*)G|`N*;x z^M$=>g2&8ct*q1ir6qm3!@cuU8#t}dI-A6dGH<%%oK#p^8h3ZsDk3T>?diyHkKt{v zP_@@9=I#@DchS(jU${=>VBzD--Mq~cma9W2pY?>Hiuaa)i@cJ2cE)wMmQOYEqfSni z+ewB_-{)xxaTi&%K=5g8ZGFJT7W%X4ZRi#8i>qgm*j9G7qj_47z*QR^t(0mgV1&c1 zY&Vb4N!(`|n`)okb*C;g_3`mxwao4rvq8G`IUwuN&#+0`i#*@7lWdGFrfRsHpX4{^ zTCyId=8jv`pfH-r?elTJ;gif!!+bsI#RCy>a6@MF> z8%yDLU0q$A{cTMTF|n_>{Sq<)84QnQ>~k~K*N3x(dq0%JzZco`yYs%U@8!#wRJa-U z=wxq_tXG6Es(qD7y-a%fMe?&B!UAV@Y`|b;eO)HYkm>Sdo;6u4a$eISQ`MJ%QS~t# z=e=HhM7L*y6o2=Qnx8-JVM@wr#`fT#>@OYd2>jxBE(vXv1R z7dHZT=Mg1kWzp9BbYlQ?a#)|X%fY$UJL??Z!0w@2KRh^y+|@w?xn#DnO{P4ozGJ|9 zkzN6h46h+Hb7Zv2E*h6E)mZK}9JJO9oC=D}3?#qU7FwRH`_vhQ_uAT;8B}t(Xg?D( z-^w^}ym;YYv$)3g{rh(S$I^z|__n7!1VKhFw>~7ke*0eI&u?eF!cI(%ngZ%ojtSy} zITDtypX~1Tz{x_-@7`U`FV3gt;`-Fs*m)%$VEBB)UFK{fLK33jZ{J!Wxf^n)(l*s? zggW!JxVW^Mx#>qgKfHbXYrAtzw_iAHE)8d^oW||XQb}Svz#GA{y1WVt=qV@_`f*Ot zg32lvE@hSZ&RxoJykDpo8RIs2K7?{41o-=xS5Mm4q7$D!zG+8%?Rx!$26DtITi z36T01wb(;!Ozg~aTDnPM{>NGCtg+~Dq!m0MV}OX!LcIidGzw9uvFhkxH zYFrYLkSwl_TK|ZM2+394cH;5@&23+;#1h=X~Axtg_VZVK zzEEquO^vvwri9#FNn2Y?dWDNDOk+d%(D&f_;XhLoftQ7)rwLAg=aRduq$Ed6u>zpG z#jCa5h$Is?0+axi_b0IpA_W6Oz$4Ly*Kgh==zjSQ>2yqSZPp_$u1FY!w=`b7?z;O* z&)D^l>dl)r)0w;C+S<4vHZ|AR-{Ip^%aA?&us)J&bKdwyT3Y(VZpz{wy?n@bDly~g z;X$X}xlhAK!QqHypn5(B1nAZOZVI85iGp(+mosbZY;8{y3@Ge3Foi@#F@7KukUi=% zYm+r@zHm&kp0q7$??63`6yey~XbTcd%*+oB>s9d?PH(xS-dI^#n_JH0dWeTyeX(OYO`NMw&PCoR_XJ7Q;Df0Poma*b#0EpfZ$;OJvI7+SELQqD2sGl=+s`Rx zN-lh~o4)qvKm8yv^&Rtig8!%Kb2>J&GRzvl^|2co6nD<~HRpL+TB7G=F85>;NpGJjY z{$v8dx8_A;a#WT40|Fp3!G;th`EBV|*|5Wn2l5Tq7Vgu9w7~Vu4Kw>Ji$|AYmx;eI zO7t5b9Gk4$5v;*0D}3Ms5TK5IlsIbEdh5jd(C%oj<)3)e@yKthGJp30@78N%J6Y=&pp( zL=8(q<Dl$%;pFkKinV$T@~gT z$e(KKen8&K7|D~q#DA)q#hwj|!DfF+;PK+KP4*NH$dsQRIqSaqFeU^0{FTOpsxnq+Zu zJYT?ZvFfSS?5vP-$uQ|K69*ZOtKX~8XO<>wpHdY}CZo$}kGnQ5-Xd!vv1|CMl;6X= z2Rhj&Jp;TS1)|uS;-iu0-qnfpJi_)&^DStCE&@>sx%>RLB7bxWIG%i4q?g}4Z@OJ^ zyhC}kDsymPt3=8gRE@xv9%&AS8-yb7GvtcvdV9Y?ZlPFVN@X8-Snh_nb2;B~Rch$h zUodh{Yx3ip!Tb^fgH}fT&gkfEuTBFiE5)IFDiMjarmUhVflRqfu@^6{GGawNanpA1 zNJvfXs48bs+8eMo9xA5dwB`6g9oV5jtfZH1VaJTocrciQ_jjUpbNA1{7dTPXBTAE! z{fJhtBg9>8@vg<|AoUtjtlKh5OKZyUaIRr>@OgKf-%oHNxxy^Y+{k7ew!1bW1+>+2 zyGhy*5>69#5fmUP7!1caU9*@u&+*$+FSM~;3}>THmr}oIoH(_bZmKN>rpc-)jZ=vVyHAObNn9u6Q$;`g0J9E@?V8VCQGgQ0~wFE9@ZK`Mh|eRz|HlZUG} zT?gTycJJTtXYu)C9qvDw^zpZUzn=5hM>^yFcZ~e|_pULzXfOXwVEzf7WJG9p|GB+? z|HVd8(HnyQyy<`Mdmrk(|8GBYwf#T%Q&`j(=dGi=mt2?G0kMhnG)T@^om0@BBTFBw z6o>Ch>>H1T#nva3n;+MZ&Op<)(OjSEW(8^)%jUkZ74KAGyidyIjA@|25mkJ016qT( z1~MdEYDzMWl{;*XxA;EbKXWr%HTspUX2^qqjIZiAqAqgLILq9aepQZ(i&uWy!*F>{ ze;#@izRtIe6i_U6M~|&=uKk2t?QUdb1TWGCx@57r1FXZR86&QVdQMY^ZmWCuhqEq1 zp$BB-J@3C4W1Er)t<4*zi1#Mjv^c2wmztlC1`lQ`EUOU+C#Iztq7ME%2kcI_0O`1R zd_-0s!{u@ZE%EoKQ=_&QfW+^W__Y1t-uY#s)hzCH4 z2R2=IIP}$Flvp*Ktl2bXb2nGEd!%m~UJ)9Su@r|(DCf?vqk((((O`dgFiTHjU*3$% zeJ-i}NU1};+ULGN?&)<898E$hg};+z?5n! zbwp81R72YRT2IB!Ofq)Fs@h;%V;aM0lV|o2iJXNB-SyZF`>DwU>SDsyOS=xfgs9)} zW}4hcU>v`DBd>E(7LWb;HFEXrH27=p)T+zfmfr;y6>6BQoypkQ+SvgR&a|Yzl)K4U z=p20*%a^+s@74|-9XFK^mNdx!DRgh{nzy38CYQ~S4Q`KDa2{!LTsKxys@X$7E2U(n z`${FiAmi>KUBy{!qC{I;x{bNCQH%b-+1Z(sdlaSyd6lYn`(i&zSL^r+Q<`XNP2API ze?w!Db(9}I@};bdf5N^H;Q433N}!&Ss2<@Ns~jA3)O@ZQzU`Oky{ z+A#&MR7sEWk>A}vkYWfamFlchGm4zQ+Bst2bv}rwtGf<`+CXFW-GoZ}%!Nv5_Kk@p zV`68&T37u>C)9SiCpPp^51tlAe%2`)V5cui4z#nEA1Y&7^Hgh9r~YM$mwq=w~7R0cMj8M@@~Oqnws=yenMk0K&|%D`=)ZyHBTBz?bW^x z1*f;H>(R5(>=w!H4b+^R9GYW$q2LXGU7=bD8g8T=zY>1L8Jm_zj7hw=TB-tb6-nuMSrez#H4y zF>YEFKwf1QHW@6m(+0klvx0R24rjOB-x6{?x5?U!LG|p2afU(6z`?hV8M9etG5?)J zO8JFc!2Lszg@^G-VMC^#3Kc~wre2BF0KO_QYyEgPw6S?~$&j(z?$NAAKQ?lCc# zo+>iW;0%}5=|m15Z($-PV^ZHc6k0;P+UGQqL&`{q_t(HIhJ4XnqM8$)akO`bB&cc; zQe4xC8a&U42vMkbRL#cVDYwciO@_f|suix+TU}c>#;Vxh)0}J}$iW9fkJJ)|v@Jp9 zSp5J3Wc}2TRAn65G)&4MB1&EuC`D?D_!EnE zDIc!%uJZg!Wa;V3nR3Z1v5`MjAq*MY8WG#9yBr6y<62Tm!1h;R}R%w zp$>n{Oc&+vZ%azHam6yPwCj-6_}40-Im`eU&}WF86}X;y7GN};!7!3DgJ$hh*FV@J z9?8wuk{`!;>ifO4x|%^>pDKN?i9og1${SRe`ugdxtJnYPO-rDU<*}1(|Dn+2N6;{p zeb)myfzQh~IR?$j{aa`+UL5y!CK}cBGsvdlz-*yen9L}K#J_)EVzE8+iJtj8S{obw zwqcPGWQi9P+YOD4FO^bqb9+wm6G1|`E`x`s8WU4y&rqnYqZ8jh@4KD8I}}bxLZYQ3 z69h0}Izv#(`d_FL*=VW+eb~OcGlDJL-ygGhlZf>=b*+CSq}f+0XQQBmPohA5dpu_q zeh2x)Yc&$l4N^6wVrMMR@k=TYxANyN(sedHy=~Cl)sCaT$CuP_adZ1nI3Fe0^YPRe zO?W}Qf*ytFgHz^7N8%iUrjcPB1~Un(ReBdluM+OowdraE?0J7$`*gU@zOIj0%uXjU zLHfpeya!KW9K_-;pE)JLBr;qYyC;l}4v}FIy#1u)WIUq#uT)h@)z#zRLI0g-gtvI2 zA?=WzzAnj@uHNt!s>Cbf+wo{#`vlrL+v2c^hFUL_);#Lg`1xwx9<{G7Z&#Irz-E$e zVdoV3WcSPX#lOjyD=GiyE8mpX=T|6B9&30M4!0~uZS61jtS%0OZ_PMRZS3%aR%;zA zObMYwzcZ@igOkl@&7#8xVF#O)ox7oCmwyQ~gk?`eG8;XtsECwv{WCWs9y`~w6tI4a z$KD_GxvP-7D!+^2Z34dFrZP@z)pa2{Y(X}qJj=VEF#odJ8>Mu0KKzNSOF+qWAoJjz zr#=_3WM3c7(I5Ldq%0f((=I>0*J)-Eu*63keYD&hw;R4VsvI{v-V&$-wa|3BEDi#R zHPqgrhdpz15v+Q|iK2G~&K6&bh}^K+wSuCcQx|E4Nx~Ebf$2&FuO4x{LPje^3-&-& zB+Vm{0GT6~vjf`X&Mwpkt2ry+rVnGNA07Stqvg(fsG&o7A=C%L>PA01W!9QpA=M*f za()U+d49)>?_lkZ#4`$kIDVIuEP)FHPLknVO3l5Ouz}11#zY*n3}Hz8&PnhXi^pT# z7I{XsnqNhUYV{M(+1ck(se~L^&0f?k+~VZo?oD-&1b!f=_!7o(36JK_U)vt-gp!+( zRhJWt4^sX-3=q5B)r&Dfau8e+ZuoG0$iwVgTwFH9*tf z=zG~L#VekPT3R7=a$ix;=r6JRAY}EHX`x5CI2rSLj4#+@LQ7L!(^#-D!r}c)Q>4}- z>k>Az3vr70IIh6ruuv+PaDkaDzS@7SJJKsIjxj8Z?Txs&F7O0!|F3yox*;_6B5NZM zz=~?Y+zmw}Nplv=pa4Owu`h2PP4ygsD;dYz+%(3du?jL~zTLzf2=yoQ0DiR}66_>2ftSdc;iHP#)b$JfghP7Jrg3(a63n&DAzdL@&&;;%{ zC5yqncB#CkqNZL^Q2{||@$}@=KZ&TTVo?jRqvT|uxZ5KHDUbX2hp)s#eOtY69Lz6u zc@%9GvT%k_n+K%H4_)@E?15vsVB&Mgby!>R@ zE;g&D8IS$kR5m@;+mN5SUvP_z`(nU%)YegxG~t3_0k2I>pXTRF?jBgSG*jCkK>jtF zUHctAlzC-#VWF-sPiOFC?QQYu%G%nKKX!oDwrAXr!tey~g}z0$aA{<+%*y+W@m>2F zMQYDnW^Tsvs#c+QH6MJk(TOqX^1WpcAQB@}F=VJE8++>>>|-5^U`24bw+p&M}%xLW6DTb50H;Y>QM7W8iKILWHB_DFN^4E<|ZdD|_AbT8-~$vbpM5 zU}QL@i)9B=2zce^-<>Hkza=P8%Rg9bB}J~UtSDPQOubJTo9XAbcmD@$|F8`i4`eCk zUqzE+cxs*Hf=|%W_~r_bUH3q9{d%ANtU2gP-59OS&!33n8Iba&A3jQa{?s%r3ml|- zOl)%GuKLFXK~7Ftup-Cu+!?&~KNxOI`zyHaT2sDC760%OX^*t)eZtVhicYI ziRl7nGdU7*h%RiH$)IQ9;*`7^#Rz)qGxf(fupbv^;BQQjWiefw<;YgqgMMa0Vq)pa zu=}We0p1(-O(vL;*K;ywt)k_`Uww_*tLy(%N!S>$C&p{R`#l z44>}7iQIqB1z;Phe+%Q27utt1JL4jqV%Jtz(DYR}he5lC9nJm*8c0VF_Zn!{w(#X%5=S=}}5t zz$4f>f0{=2JLOoknzCR&di3Va8?6P)E}IL;8ip1R$5IyocE^EM(7Ntn>L9j%#fdrL zT_M*idnu5eYMGm-oaabgf@OMl?f6Uc`A=>Yv_f8~9*4VQk5KF)<5eT@=FVo z-wP{bPt>IAKpOCSa4;lo#V~Yn{48?vg{GcDg)Hq=YRTD^1UJy@Q-o8uLwcX@%y0#9 z+KA}^t}wBL^)iO5=}Vq4thj({Smmjp^-ATmX@i~WdW7HCFqPgdt;FRP%h zxJydO0VZR0YLrDN4WwpVyr2}gP%gLD`|?$qLPT7~<1FK1389?BLAlPhIk(XY{y4}2 z_#+})rqcznfKO#iZST(0K-zV~p$DIJ*RXrOMRL$brL*%Vl0scP%6zoMhT?)vy{Rb%ce**=qkxV9xJsMS-8!LB}OFMG^@5Br4it&ST zgsL&bZ$~6m)yJv1pB1+RQJ9?nCHlYC@g_)4oZRDudh^cmPgOd<%4LF+oi}bK8bBnO zF1>_$>U)9TeLwd@b%29~{tfKCOvpP&sYyvsD5qEt6USdL>m66D6`N0SUt@^q_$p^I zShIq2pG*RBN7S^qgcqOk@^jv<_9t`Ntx1Z7oO~=RL*BR&$LHn)Asrab`5=LJG8rLU zcaBQh?Hs`l0L7_T-6Z#+z;VyQf-a~IwA#i|xo7VrB&fKleT=UB;Iy{HhH+7Pnqj@( zPN6DpwIQG(fiL}3rQ&l2zT94H&_5a4=i>=rVwm${Z`qvTcZ|%`*;ALNm>8IJfx&yt zcjF-?r5EiL0bJaIx9j{{5|g9FF3%uurQ+tkQR|Lg?mnMVpe`vHPb6>y2(lYc%!Qf| zRq^tnO2RCn) zZr`Max3$z0`>Cep$g8KZG0C;bxu=5(Vrc-_(;x5|l2yyCU6pfU#E4;|0L+9dP;MZ4 z_mv;M)^Lm5;xXe&LLXMn9TQqmG5B z)P#lm`}^NWNW9X}h}Re~(*(l$F%?z5_F|8s@-QU4o(aM?tp7~Lw-Zpo3+N4fO)ff3 zx$m$wsRx6s&u!M(0rMCm!p->TEC%(XL%h4|O!nKr^~1jT78Hu~&84;k>M!$|^8X@F z%8cj(??mDW8d&s1oynN38c5E&F(|l04e{e!n)`G9XDFz6=F;0~?{OF?(20=QNati- zbb!_m<-wQ0K&{1$_)sA%v#$|RFwvo5F1Z8BR$Z_VI^%e-dnbb+MOrF5`bzLDh5cg0 zDc&}F`vpK!gzT=YL_|dJGFA(QgtWA@Ay40bhbiraN8xs`8g2H122?BUW1rQei3jA? zJf@|MgcKcWPVLk7Ax_RwNU+|9(rH*;oY@7&b3)DZS8Wa=033X9Qr={bL~+AX>K*6b zvqq{`=@Oyf6veU#D7|EATu)txN&^EK1r9bE0%IvxR#r5BB_n$*+%jztM`S?qguLnd z@6t&XrN3}A!P5G9-A1Wsfo3C?7yul<_}ry$m}nr~_t#p0#&pzB=wwnA<+R|4qmXcJ zfJwsq4GySCcQcHU1KvMfp_E;np(X(;c~Ht}_dQ%Ejz}PF7i(xH?^MA_!i{!5Nc)4j zafha{gq)6qGN~sz#@Zi3^(1Z_9XGU;QXIX@$x^NMNlJPwF}AOFXne50pWuQ`VM?Y7 zNM5Q!*c+eyM9umS;_>XC;=^)fr)P&BKYHYy(xuF7fBur%pF{!@0}$GOY0U8Zlk$Z@ zojtESL*%K5h4)P%O^O@?g9mhHN@I(EN|L1?uX@Lf6=`(m-L;a97Kkky!p_Ds7O&E((CIgB;h8p3QHLx~vU z{XKEbkjb5MTJAhxP%*i)9uXZ#R@3yWouG{E@b^F_8*B+72>qkWWRICnj2W=4o=E*f z1BhhVi2OH&MD$!wm&axObYEyJWn}~)Xg{kdx-#KG{Mh+siGtU)b?<(rXoLUVe5IF(o?k(T%MLQ;BqjKyE<1u;{9}MzWKTN z@5gJ36NN0g1h<#Ex{d{xAUb5Njp8O!4{FPgZ^aK~$Wc*JUIBX`grBK)v3FoVZ+bHF z>8TM{@!B{pX+<;$4~z5xSp~zT+>z+o&c-9R{;6wDc2*k&JT3(!@^DBHAq9G3wK;pA zIZ&!tpwFlDr#5!kJ?G_x)34nPdS3O`=!BDYa9q>`LXo3?)Jc(_qp%#XaFJ5A$=wNH z#Sx21H}8j13-^&%PPt=T6?+{~p|VeM-RCyX0ZgObieo5qeUH$L$DybI=4l>=K#9kv z@aqPjaH>|@!;ts0JO>5KKq2rvvqnqw9)$|z9t4UVFM&JlZ-sY0R!g&>M)2xp@fMJH zeZO*#z6bujzhB)GDEB#ez@Ws6%wAI1Und58%_GyP#>k@b&VIx6bl{gCS7DA3(j@lU zxsLUJ5QG;ucmIPR{Qnz4sJpFSwt%(l?=?6IHm} zndltQSy7OtE|Xpg&lIN%<6LO_DfhTaYR+_)O6`@7arT!nwP?_r;R`fhR8(;m%HL{{ zjFBDVHyO(u$(vhdt3?BNV5zplSW9QS^Jl^%lynTBChVl69<$%XrcskjeDu^n5M+S_ zQ?9m1_RUD?OdY9{@=$)VjDiKoz8Og@ZcC)Ve#lL9QD{)g_W*vf3qRyJp)<&x`~GJ}s;;=}iIgWY||h2GOu@ zpd~0mG$_OlrSLRl-e-fqmY&H`x;@zzY=2}Tp$5J($c*5EG*5P(7n2#tTNKt5F3VjO zN^&`_+!XSgKJP&Pm+_g#LJ=!sp>)!x`(oP_gcfQH^c@QBf=Kq4BCqlnQuW%mLD@{T zXP^gOCAL21!U652q$D;Nkbm8KvNPOqx})_NvYhv6Qk07hUAZPw(1WV*&ZciJ{_~f% zy`A+@D}VpL2J*Zz3rr^?f>GFj>Inr1lm2%Tn=PUB)^+*5`%+drdVAL9!%~|a>Xp@1 z^q;w@7|he&oOYuI*i63@MT2f(s(L~>2Y$lc3X@W?no9fa`mwRQIy#TgCZk#9-#A=ODsk=c z-nGZ+h(b+Rxv8gy8h!2NY_n2leur$dtH0^C?B1KVZ)r5gF&8ajHGahE zL-(*%(%4OlNmtWf;?o>O6hWqp*ohDvY^OuW5>5=OvVukJeQjtWmm zpj4~0)rP@6NSUbrUMm{NZ45yuQkuYQ2RN_+#$rNsWn=n-Z{hdZjC)IQFb~S0jbo8C ztC**Gm*SS1la!i@jzdP)P)b&*tu)~S3=DwM<-C9p8x0s}uS>0)!@D6DSb6}fp#1aWu zF+=kR8l+_pq1j-I&lzQ{z>{DFUM&7K2H;X;vk&9L!bIwdh!iKd=7$IU;A|JPS=m}B zjBnX)t#v^Xd*$;NrykSZJUe|mcnm^oKM6%{^7HYL(TKW$+6(#+gAzLWs!jpUYEP>F0(f<%poe;vLoEXKbT|C!T~xubi@kU z2oRHz{g@x$-|9om>@98Zu*Q~VAq9HnF_fk%9VlSagM{*_sD{Uk^&7!QsNhuMs|K!?1qjQ>}h z3Tgmx%G#oWhU1*SYh16*d`Jtm?AVO#z!QxN-%W)1)O9 zr?IoWQ>5IkFZH5pfA{3+t=p%FM}5vQ*T1h(cESM_ZlTk>^Y1{i`MP>sT}lFqv6qzIFJgG@Z^oaHPaJF#QB;P@V;67Aw) z9BP#93~fg6!=@TON|bJ3S9_6TKLITDa|TV^XpWrao!~@xH?n#cP?=CmL^iJkHx#*Z z!Ce+obtSEz;9wc^AcX-8OCTwsQAf ztNW|oTNEU)kS`2fCh}B4T!kq*78V^Kvx^gQaElU&_>;edhnUk<6T8~kaiF6mD77b! zc3l!l)a`RIaN%Hpxap#}E#1lJz|7EHLT*1{TpfXv6Tu-lzvuJYIpJAu|%= z+i5Q}fE{X(K!`^5V}PcHa>d}g`4Ni;HT-;B7}KgZE2T0QF{p zR@Tq|4Tm+C&eyL$(9*;c|g4)*1yuWpSsM1!^i9!@=kCtCF?&iF8T;OOa42D4QI zKsN2w0#Y=kI;gQxjNO5#@e^l^-Iyf7h1o{y2GB&tTMu;adkw1>j=6ljlP_Z{@?- zjOKojKhZ1B&k@~@6L>N~&X4P3Hx;9GX1zZ0@L@L|r^^lc@MnLH1EQf^|2BS< zl`b$8;%+|D2o?NBL1}eWzpcdfSYt=OR}_@8B|*q!jrp^jf&rQ{0!=K=25Hf_&H~RT zY1oE@s{-sG{Dt~obAHQoU129c7^ zC?X>28$j(pMrokSQ6^veI=lI#IP@XbQznd#E7MOHR}2pxraA>rTnvB^1}&5zXx)Jk zh}T*b+|WT!3dC2Bi{ZBV*h4S(P;>lXM0#dzuOGU)7#J9OTAuxjI5zy^8?JMCxOz0k zQP7h=PeUu+DC_i@*HBQveEM8;oga8TxWJ!{k?cKXOq*y#6pe)KpLpB|VtDXwxXqIw<6IyF=O`)|o zn~8fD>}qM-b4gi9tFxyyXBp0MrV5SC$aFRv44{E4D|_m`%1T@Y{;(tt*RPniTJGm6|Q(4hyZ7}Z$?BuB|5 zg6fvw;o1-w9?{_$gKHMuM@ZmGbG;eN!`LivIG7vL_l$M8t%e zhMGUwyFj-@lN$Q{Lhlq+l|gqSyuN!3cO;5ZrfivBU_#^M(Ytfc7J12o&!8@*t9#Gq z6ISTq->P`Kch83U5!17v;?L={0Ao>Es^bJqMZfQ8|6cEXCbQ_*-p>E5hfB9+$W?RO zBjFQdT=1TK!Cs+LxQub~iqy)2#AF9-0g&U@DmfvE=LjHXlTEMfRdX4w+^%JZEo56v zV~4p1-WDb%rXq5^r`;Zst0H6x1B3!?tzKz3AniN4$^gax}%)flTF8LO$7VU(C&)9c_}2z9CAI%g-z#p z`T}laTX0~?WszGYyVDkjz(;Gn@aoOQ&=ivjScX-O!GgN7_drC;vvA03#_^vDGnEifPJ9%)*H080nRrL zVC{gkCw@dB$*~;=q2+6NITwis;?1)@4iO?t3cmjGt``SVLFtXNZVUVf<)``X^(!xtnnEpa#uJ z1`;$tt!3O5o**OJy>A{OUh0@1&F~K&u1QrZ%Pk%sPr2;Q)(bq^+xJu9Wx|EDXubni)r z>fN#X49X$!c~3TvB>-pEJ?rEuU&4}t?2@VzPnTbFYzIN@Y>VWVL>S1Wh#OH|8))(i zUjarLWWSSyRBX>?41ruaW;$G8YC1c`=sLvypU$p49Lg^2Pa<0~gprK3EJLOt%S)C> zmW1p}WJ@tg_7G(qM4H0bC1lHPM93~XW68dkMoh?-Eb|?_*LS^teSduaoNMN}p5=bd zInO!wIrs1W{cvI}O_-lc^1CGq>PT@tZyB7q*{DN48K~2?zT7tM`CjY0?Y$xNql#0a zC+#qzM!m9P5El)X4+JUmGP}bXo{z?_gH&Cf^#Cto;bi&Vf;m|4LuC}hA|sP$yAF(j zLyp+AdH66G@Q;ms64q6^n`;a5z^}LztaR;^W$$>95QjU{39xx5co4t|y?+nT9cuZ9 ziQ(VaC`79Jvox82YoNwRYQle(fXn)22Zsibd%T+y^btVDVO+|YXRb?FI~d*w`~p=| zfqkci(HO7*c=h_VGO~qV1rX$Wff(>(?rZtFfKEI43{kfe6baA z=0vg_>>7abNRz8t0I?czvaGt*cHeK`tPyY<{{9ggFY6TA@d>=Y-9ckZSE7O;2}ZZG zI1D(MZW_e=y=;^Ccfgn=Jw5DNOKV!>0LCBS6WnXgDXElly}uqu0+KzJXZX^r^ zpc-$()!YE^`<>K;jXhU_pBq@{fcB0NP1yOK4zN(9G5gTL{=sNR`u3Q~{As~W!(Z@h z?G&KliY<2@zVW>!9Cw(2IXWIKHC;Xjb|Vg>ttWy)*Jfr_z}PXkke-pz)CBZ}|1#Lj z?o|P>hAK{*w7590268V_3-Nnxz54@$%@@jaA|pD${6^W6+5n&t4~Prlcgfe975v`2 zjN?G}M;-$Czhnze*7zc<`TjRxboDS`q@4C0*?iM-8zmv3&sB5s`j_Sh46xWL9gSG( zWPr||=}D>XSYhpC(+8A2v9FZJ;**kcugW}mOb+-Rmx;Y?&HRRW=$F^h*GoR`+Y9wm zx}d_cywW}&a@vDk1K^?6e}4g%YXsm6b}33p#l2H~Q975cx=c6Pno{kRQ1|$DYDa3U zbDE_)sXv_iewWc4`G=ua7r)l6TN>Kh?{U%GfOw3@?h9MoC58^Ke||SkbR~l|L~Q&t zBiGeT1K~~Q?1jYCs;Z8+hW`EiMr3>Y1>jm2xQd;Zc6`OpK&G|Lu^5lIC*m4MzUXR_S(Gt#7k!Py@i)T_^U zcu;REzmhDk7SLDV9YqlN&y5VA1r@bohG=v_HD{qYtFcgyA%c%jkMe`r2ao$Of<9@^ z(*jx$alTT@fZv=gu!=iJSe-d6P-&0c(~e8{mU4fptH!P8s>F&Lmm{775lc7AgiWM; z-=}5bbJm!r9<(qIR=w{Ffz_gV{2C(GW-l07yq-N-`DBqX`5$wRB=rl*ieA#ma%t(K zJ(IrMGxKsVksyjII>m~euUts8O&HnfMuKFj!b+v7@qC`cN9XB)vNC?ak`h47p}GF$ z5%8lG(fhwZT#k@8gjOG^JKs*b+gIaVraA6BT1qwODCtVp-h#nk==bgek}rQIyf z`Qic0)ILHZd z6=4?GH^7h2{hsh;-+2PRF!LKdo;U4K_HsH64Pl>W5)a-|>*)vyfw<_y^Ly@xyGfgsofHrZ>lTA@@I#Ls%?>9nW)8p zDBXO=kSOqwo8AR%L^J#pzUP@{E+s?vj1a{H_Ox{#7EDWoL&<2AY=R&1@#h zHV;_h6LU@PBy$3`-yMXL1QN*&fY!Q*7g+B#X3sr8eG3?PECAXFNq_Mj_4ssf(sx&( ztruAjhICNL#gF`Cd^U)dzoO`zuCuWvz8lq1jEl1=m}aG+42yW#vOqKuiq+iP+I5+z zBP-pXwXeEk-FIija1;YD)hYsSc%Rx{7{jA6&+usDLI%HiEzlbf%lUgDpV(5V@7kp*7ILpt)(P?gVtB%mI!XqOX6}{paAJE=H4c9k-f`3(7xim+4lbNrvh=n@_K# zQ$$3>1{b$smH&;05rGB>)saw|a)$v9pyGUL^P`egAo6o^=%3&VG?Wx3i|pjI>^gjV zPfviBzyOQtzdV`qdPrs-NmWrc{;Q%u`N@x34d)MykGE=dpVSwJ)cdY@VtTtA{7hvBI0m`luQIaYuTgd#wdXZ{Vk{h@tl?QN(-OTs0WKFpW3awtz4{@h>;O;sNx~ zQZhs*I5Y&8lr-i%`*flvHp+5KY&AumQG{Zj(AGgG|H&(;sQ0v1iUR#hv0q`MrL1%z z%EJq5Uz02@qgxHMlCQ7=Y{a3civUho{2>j%*FG&1hfX`aBwTGMcwtu1I>XCtwbuXL zq@tOTG=>k!%3?3G*KVSyYhjnR=9`%OO99E0rF-?!+4-cPAoY`Ht4v%E5)k&ytJYyU z>Wx1or;SWtEXAD68e+*7yS7cOZT)=ZvW`y;-Y<`bedh1=i~~ibpcy1U@(9@FG-7Rq^P?n6XVYg0i9ZM2{FI?4N{%DV$-a~ zKTF=EI}M+s%{AC|`MEF#Q&m-EmpfReTiV#C=A);-9*nVB+LDp^w9z#5L6ow{@^%(G z0}}^xih{$!>!rhu_BLM&OZ;r^p`9ov0g?A zD}GSumC17#IgEUqHjil37tolljRMA9d~PCy#+jd^6Zd^bFaFTep4 zCSuX1*FEpk#ApJ)Vr!+2;Jc#TT`-fxuixc{C^V#sVIUA_?-F22s5>Odx z@4&|KD*G}6a)pWgN}r0`@RJyDSaHdRR zU&RCtn+DhUVtd@-?bpDL zK*?YeaA`pOmW0i}m)BCdVWC*rZ>XXUK~J@Yz?uJ=s9G=xI3fpcMHd)HrgkdNk z=^nclIiZ(NDBDqj`D~=(BF7qSxYl!G-+K>qFeYTsll}UpcHeNPqa39VZ72g=&w8|> z++FleIg5hC{rZlMQnq(_reK!t8*ZUZi#%sIo_f16>J#pGD2!2_CIf7CcWs3LbJ z_v3atD#f+6o*$onq)ve)SW^Tan{-r0o;k~Ne2b&f&OIK&`=6^b)*ES|0ZNcqO+$Fe zzqbK@r~;I9$E}b4522CC5R|mEilwE3rlw{d|DiY#AKyFUM9nN^aICAl+fk0?=#bAX zU0stLf5Htr@pCa9HaSDc36>+H2qL$5p!E|Ir}?ls6)H#1&l}FWctZICo?V~+pVN9GuV?{B{U3Ag|S zL%@Jz#{ht1M=!wl+2axrUESL^;U-|n4gJ3^9OnTX(MdS~0E6}Qh8yTyx@C#F^yA#W z(_MQX&wuCt$@fvedt*O(2LMJu|K$4r)hR{?M<4s6M(>Z_zTQXRN5VQB@f^;7s*3nx4-~)gI3;;Su{Qpw=X-|&=0O0B_0B|DV=Q%q906+}`0C?y>&xzy#0Ovyi zfVu(CyWV&I66WO5=W!<|0ARZq0AR5M0N4is04A${7xd@7KYj5pbiH(RkHC?h9!D={ zfIGkua0vhbU;*|3*&|K?AP0~KsC=IU=mJh0|3N=KPaM%nhLb<&3g_Y&p`3o0VFEE{DV`sa-e#Bq+(aEtNcb+(T=IF)?%#6%OXa6VZ z`yT+-Gsg~3c%3)~1RQ5Qc7pZT_cnmwk^4^k_&fIR766<&eTL!0Nyg(pE}LBd08X7a zcJd_Sk+_UU0vv_^@e?OcF`Q;S!^Y0Rb=iEFlN%@xPfp3NR=IBGn>sW?V-!|UHN`1j z)w%0c%Oj#|;qB*NRU@kW(7vFkPE1KH@u*vYBe>&7m_I>(9RBF`%+XaX)+3Z-#~GOz zjx2WU)Q?6-psXj^IXF$>i2Nb8%km0}UWxc;?@rx)_+ohW`g@?Ak`5QQy@PkbhwpU2 z*&`{Bvz}lDXaf#Ne^=}OE6j=ZzUq}PF#%v8y(FhYe$!~|U?9M?^=WW#fb2oB zJ_VKXWV2YFc7Q)u2uFjib<=?9U@A)7e1+I8hKKeYM6`E47QR`G;`VLkO;AueD33eo zM|9Y(=ywuc7R^A?0v{1P2OZvRy-&uQ7M*mSPhh$Dm!xW`A=+Ls)assXGig`%erE@n zVzcIPlS0%8N;PxguD+j+vlo+y64@~BvDl>WAgvb5N^#SA##swg3~RBj!dCaWktX9E zChZ949{={JUi!XF{YM=w1BQ!HM7SiJR9}*u z+&P3atLi4?r$D05p~f5a+^jZModVCbx-~xzL}H{nd1p1k9ofjg)jC^$(tLGnM3;6i z)&SQvWa*{pk|NnrBP=|6Hh*+ig09E{`3V^X9<54D_j}OEV#6J_aLy>12?p8!XoYuF za8Da$)advau_@Zg2%b$YfXBq5Bo4Y_JT=586-v`Kc9LDx7!FHsfwUIL7~Oi=hFw4B z3TZm5log6|&?_EvS8^HYa(tz|qW;+2N9N%E+&RZwlPNpu!|bON_`bOUX(7niY%21+ zo*7F<6LXeH4HQAi6rmi>|H_D&xLNg1?333Fw#mAIFVPbe z+R|qAY>vN*t;jQ(G>O4dT&LXoA*N+zxf4384U#;C;sY&}OdC_XxesifCaF`TV@2N# z5+x+8)lErP<@UnS)j|t8rpLqU)vrP~U=~VqP(&>xA43qoveuckgZ}t1Ps*jX4ddm~A0zOR0BF%16sMA}jP# zjuTGBGD5s+*@oIOzAE9Qdm!pLRkYt-HhginzYT*GPG~vY+q_^u@=L_byim!$ZeWsc3APu|>EDEMD*v+%OBf+3D~J zB|PO8O=}rvC-dnV?EF0{>@Q>V1FS!S9_3HdzBK+>G@A9$wNxKPP}@WESHd2p(b6)s zBq=Udp@BlBGYV_8ARgX(wncdIYE>0@uog-LJ7`6u1^4E@z!rxRrK1STH*f^DPk5-N zf20%yjXy>S_)#&Q{HRs(iB*k?VbwWu3&!((Zho=0&xs>^R2Ml5SXfsUV_Y6dV%n=a z?mnfGu2csGJ52BUU!B{`l2Nkw%A{7W0{K zgGCOO;jNwG%$kye>6;n3mbw;N#*~s??lhU5qHCr!Z&t1fzZ23{DX_PEwNjZcKq* zDky+v3P22*U3+fTABxB~BhwW)8D3IdOLHw@%8JcoX0kN<1`!#+%!i||qRmPXQ~6$_%0)6c??^h4ucX;Fa=}NlInBiq9Ms#M;@zMk%7#KLKIg585TU%B znbBl(I)VEcBzEKy4>oXLGtx59If*Z!`?j(k% z{56OP^lIhSw{yGCHm60-+P3iaDF>w{%0?AqK3l{TnIzsQ6FnDz&YGo;a^5ag#PrWW z@G;4HfkGDT6Sj_UEQeoTFF^&XS6}wN(AKbF70!t@jx)3z<&P+S@bQxgvMg1C2|l2A z(UL9Z@$%Bop;Mk^Aq2kFJaq53Zp*YwmNmEhDOY$VZi3X4BSae8B0$~Nkzy|{azd?| z$g``%@R{%=W!^aaG@WZ76nXADp!7A`PZv=wQ-5BR(Y<&)int&v-S%j8drEA^RHO#@ zc0}hA-`X#pd+P*^^`n)))zF^)lw!U_wqys=Bva5eU{YJ+Mp0 z1TOIcOWDs{h`F8ap{_A$XuanYIH0Nms<|`yal#KJX|^z!Ue>t8of800%zysHSTtGm zv1#gR#Ao!3qM+8m7lURi`Mxe*zI=EE5)`-q9mkC7s-Tr8&ccxQV97or1ZZ$U0JHuGylSyYM= zSsO7PW#u1bjpR@SU5=rR)n=Q5D;kg0rnzN&O_!gcnPFUc^djQC95QLfwZ#3Ea>k6d zq!`|`p)?{!;BJbmgFh=;?9uwnjh|Z%oofwClndUk>O|9AZGFe8ADI-P~(&$ zTz-sez0@z>xC$>^9qfa9?l?lf1M;lZW|o_M`p*ft#rc@7sgyx9&|?p|>@JV{#rK+0r>J(f7WOV!c20d z%B+zH@yiQiOXNI@AMlcb@RkKY6@ZtSpVbC#*MBHN7kN(MGQ;1l!1T+co>^YmU5#x{ zx(*Im;DZ;drgcntIY6y~qO{cjgI|B*Q9gtj$sxsTC3`uT1QnF5@K#(^6v zw(tX;D;wCg(UP|Qp&)iu{Y~o6B;gr%Ky{dTxO`o*cbLjGYZl7m6oiqNxTaJD1E>N% zsjHOdj;w0VXcn}xXIn^fE?~2RZB(BrrCkXTgmQ@{9bO8x|=QbUV2YIg$g@ zXpu>?ol)u3?g6T=6!gbq7V5N-tP2`C27b52ol=Te5?0Vs!6+GmStK=u&SeA|Mr5lV;lcbU!ugRL%+kn@Efp9dmS}kL(rL$Tl{Ojkd?o zGNTN1&}p7Xo4YIOIpk#OZvShd=XT$L(9XHP!DNTU`_)pbYe!Sv%ol~P z`Zos(su%Ks1`t*WbAPCI8!NULL)2%JE`uSad9~NM_QIWPXNJ<(T)$FOQouDX_Zl)z zHTVt71_qbLwwt0Ia`ZoH!LE&Xa)p1QB^i@ZHK6mAeah!$2J*M$76>0T7f)!sy%jI% z*pU9HHRO+~KVMg2sb#n~7nZsStZ$0>bXiF1(=m^|oZB&ABONwd^C(Sb+iDei=;6SV z%{Ac*s=rYym@{&Gi6#Nnag%489VRzD6zIurMpC|msQWWpC?kZlPpb3ar*!WhIvz_= z5PO*($@BTmi>up%HB~IEUCPYZcStSv7~_a9CMw0*7$(c2V9+Q*6zotd6mEx*aL1u< zbL}3ghgmmCP|bE2?1xd&L1u-+HsmJmM`71q8M9-RtSEV1bWN-4;=aAze7ApryYZ$z zb)jSOs&dqZ+rx3$h}51x+jC7zds2JN6r<}?M3*4O1?skg!+4W~M?fgEj^VN#R$ay* ze7)Qu_*o0)X8s7VLY`2wo~(oqXd@)@;-}x*lJuNg#`G7tRDas=_^GDA-}`@C?Z3IQ zoP~zJhT31sp3%f<9xBszHBD}3Sp>9lPN^AFV}=+U)k-yt1W7GNqjH$wamz z@(eKO04#fKkz~Ni`KyJ~Gay=F&rZUyK(*GB=2w4tiAj;+&*G=q?TMfR4yb^;n-|jF zY-9DB&EbJhZ&Ye=6)E;4-4aFDjSwmM4#2!rxS}@6K^NqT!wLf9nHZaj*;C`fsUOE- z{)V+9i0^=Jk@vT^T1pwjn@Jx6?|MnWciPf z%gi9NF~=n?{rsEs>B-a+{<}pHc9W}~`%5LCSYZ1R;FV>J%DUKK;miox2;y3Kt8xDq z_n64p_m^*<`>h%a)ohBGD7OESeXt-$oZ3}tRV$49>Nk9VFoB~?^S9wtWR)masFRKT zZ^M_&r2YCBsx;4qmXhnz#M;7VNjjKsE%%H9Y9#egyOM^G-Q1h2zrYB@_G6Jnmdb9} zHVSZr3<>?BM31=zQ@<$!60iTDP?PC4ZHbQ+vkAG6!Ev-VJyDEZNn9`M)kLnFOTO9l zmQp$hDTSMxp_|J21wIyroDqv;p}f*He7l{cs7zqJFx>nz0M2IH+ruc&SY9NW-W`_b zw2KV5&1qF9T^o34tVBwb8HBjHpqU225RCeA;hN%__SG%y7dO|aEH{OZwX#;=G@NN> z^634E`$#J0pj2oO?|3Ogd~nVxlz1gH#LbfLMh$oI&on)uX8-kk($LLb)s63fpb?3* z9QfmNS@aE#rryD!a~c$IK=DNp%BQXwWwMcuQ8&~uB9!Cnm6{pj#5-0Esf(SlgYH&Q znckY_`N{30Zftj?1Mn(2I;hAt3#;(3`Vc3`U<|tAXH=YI0ng-_O&jf^1-0BFjAt!? z3Ie9$Hon597tKj_PY2Syms(fNjrBTG=#IcIoyy`%rdg#5)Jsr1BeAx6Z^l-Tgjcmm zYBn|;U9Vwb0>-tL(daYb;mLqd5GjQfJTcKT7@w~ErCgP!Qg_RZyxmr;EHPR<8>;rxd zQ{(0ulmk~5O_b>J>34TGxha26qG54Z*4^K}2|=0qP-N&q9vu@= zJu1beTm{1W@%vu)k(3`4TEmui$vYUaz9GrGs;e8Q( z-OyD{U%v5Q#t9vp8o{z5Gtzi@ z6rlhzDId`PMLk6o-~=2jIl#SW7VBqGs&vq=p28O?ROpXfx<<8h(h>2QKS)l)gV%!D z;n^y^k&TAq`rh|CIW{8ij+a^NzHqx6RB=#mb6eB1zj3w1t3Gp)l-KU@qEuoi6LcmM zvJVwT^+Pp{?0$ii>?7;a8G8#dJ!hpxGqqCF&*_y|`NIXAt|+#N)~XTYC%iU+Mo4}% z=#_D}Rq3cMp;k`z=F_JwwvU|`t|nADzVOt(XI9ojN=gtzz73Jp6ucpACMK1sBfKvq z&d4arnz%2l_Y0mPr6F?1Y>7azZps$7yvcg!Wy!PaL{vo&KCQ?Y1JV}L-L`V?iX`&E zhi?Y*_4PZGp0t^%xDL>w**;F}2-%oR^K|SG5Gt#jTnmDB(Z`9iQv}Y`3pra1CJF<18Bq$BGPV)O>-q&wNDyzWgo@BMr6>>n~P zEF~F=B9wMsij;^*+(t*-g8tB3uz4PegFv}v7~WrvNK(<3g@9C(zxuIAjl-u!2rl~L zJKRCPB*f8^gsL2ZWDD)(KE+ndXVxvNU!GydA{D8pIb##aR}*V%lG5zhx^zXR(^qrz zXHhs^aH%rsIh0nCqYPuTP(9l$VD{=>-UroaTlp&RVrCN^ySrB0r~b=0W@q=gAu@V) zQK79~*WK;By7fN#*%g*+9lvG6G!i3*(NR{c!U4?l!z8@kyE^npM3Hz9%ok*^rg1;c zUDa={H(XYz`p(U*j3kOSQ;}&Ua+H&L&d4ozEKZobogogtU8>2VPs;J02>w}>i^Zzv z`xUuAeWmY5__U!)u__GP^d)bb<=c=cb=i;3V$}=Y74L@p4g3|kEUiSGBS(YnF^kquT#)WE z+VW(Yn8XJMV2FO#ygqm4EH}%^2~p7k78Eg7oSd_dc{uymkKAho=6=YDc3QEcJuJq? z5OBOb??3k-@xSri3*K=zYUDN^TXtJa>%B9TaQB+8Pv)Dl%!ri0)TE)ZP_$%GZS`b5 ziPh($YK_BG>u&$CF78u8e?*OIX!-cds3}KW2wuhS%1a6Zb%KlXuSmK8&5P#KcQD&e z@@J*(J|qcFa2;kQV9MeaTtOs(phtXsCZeEe^+aW1!@TevRl_8oBZbHXk9%r4ht}F7*aPkG!qJ*6- zoXSkf!46%y65AP2ZPd&*l9c?&ROD0h%hG49<@fs^d^CS>So3boxxs5~^7i_i>tMF- zXroNoquG)qgVJ~KuL32DRNplnL0p?kc3^UW!v62Be{hP8q+W@+<~e%04aY5wrA;LM z$=BllLPi{Sj$6L-T|;S7OTM^&W$=J)zOnn(!$9T~*EI11#tEL0G$&}Ffl=!P6M+h&0w-vlLIW?Bez&x|4K| zwU)FVT3MC$L8NO^(LL6|L05&g*`?wn+akBo;x=;?ZA3UeSTvJtX3YY7j2T+p5~W@& zy5aI2fb8wBYMG@`Ic961%D_tpS|@~PAHou=b7(wK^cTp5$){B(`J-3{bfQ5^!mwDj z(}Iy@u4q&CuHb|?)eVUpB|=U?HOx5Y?Q&&1NY-AY!Dr)YIcs^X*Stj`+3?D2fmoK} zl}xYOf1Dotlc#RHJvF<(W^*_|ry48O>iK3gO6ChnV?Z6Fn-g`?88SwPMvBeS!r^EI z#i5!+oLl*Q_O>_j_bzg5DEq&~p0Y{HenPc!bdrx|dmKg0LdNLPJC{m&9fSZanOF<`65&{P3kw`!t?$xx zn-bT)W#e|(!n`~N%C#xAmj)O6BJ+^a3d>5LRuwN@Ia)z9iVIVplNzkW7%B6a=+I&s zBThfVjbZ6G%0kLeeXx*>SetsBzcF#j^ieL7;l{K`1_xdyjA@bi-DWWB#v<`KRUS|MXGl8 z(=ha%d{+lm9CcHwPZ0`h)C+X0AliKXqyFs$yFG1)xjBB96VzRZNN*rpjpPXH8{1=K zjoQGiR|4R+ny*GBXSU%et}NsB`bkcDlbD%b*7Magy}(*7QSX!J6j|?v3~SNE8lsMC zV9k?cssP-axC@%T(G{M3Ro{PF{_DWbSA(HKFsIEYGu9D{ov(VIw9*0lmhlosoxF3- zMvWbvqfLCLzJMGSbczE6;f200z3SrA34#|(?UBu3h)LG?7+xtMwcuUQ6_H_;z9!ju zT9Hu=HaeqF2gAudVRv>c3ekQkLno5iv!8@_s=GTKP8^&*>dGWzr^peT4b;ZeaK8xg{)F{Ze zne8qpir0T%p>8poZZU__{FPI`KQydJ#gbr~jZscgMKhLC+20_#CBDZpi%8FPNr ztLQYrkKHI#sOd+PD%a5EKz`=}oA!kQW|;U2`OXJMuo;zb$S zPLKGeV(fnpK-;WB6IdY)b4yXLlDlAg_Qm_)V_u&H*5|o42oIpw`%GrVqKe0 zNPdU7xVsd+wKU}4yO?sz5&8DpwGSa*4`la)-`+7epXy!Y{PUJG)GRAWc)ZRx zF(<`~D^iDV>~9|w>U~yd->cnrU#xexqI=7b>fQ29-CJ^7hYk4D64k|rW(3X66qAV(RB$!*yP(EaE z&|xyXa8$juG@{qvc$#~Vwej}Ez0+pCC@a6W)RH>e-gkjay?{ud=J8;2Vh5g12uMpX!gSD3l2VXeiI|wKu7jAE zo9pZBRl=O1=BA-!DHWoR;iGx75vgbzN6V8OiMsvn<;a(?uYk+4L9%a}%jqPodRI^v zwM=Q8%zme?_G!v1Tr~#6mZ?NO>7oeXI~()d{_-mALw}}aC)`UJ8rxATI+WtkVzMY` z{PC3H3zd7s0?$^9`7Ji;^AOycpah^qfuZ{rvxFEOw&K{E>=&l`g2Sx6l9pE?qj|v^ zv^>lY?*$KyRh#Z>dC5nr;9h=~?+J~uvFj-8E5jzQyf(b=&@E}QUgYfn>O|sq@Gz9y z4zs|KK~Ia~KY%s+IJhX~@$Szo9||FS?D5%NY4(QTrg()Ns8Q@yJv0!`ku=y>JJ@&K zD)Z`He)HZ+`&Dzz-#3_+70tY&9>)!MlPJRwfhY46Y@;g>Gy?A%$;U3sMSwM;cC51j1U?z;~>*%zI&n`;EaM_9if#E&T2^@|wMYx?6 z(w&5WtBjatqzd=k2sNDTbjW$vVOmdzfqP7wt-8(0aJ+v6Ul zItN$llGw)dZ6sKKYz?irzbiD|PQdn}sj`o6Or(jA&0hC;)UT83s}OlvM86(WSpHS=eU4wu^$7Dm!Rv3uRQJtD1Fh}(KpHXa3%Z+7u1L@ z@|mn4%8O3_GHKtR=?x03_O|)_Qf7k6#Far-Rk!vN1Hm$#c4CxPQHc2@&GCK#WKI+` zQV(n)0hqsQ7sQZ#5AN~CS5(y|e=v^zP&_NRJHwU2ta8ukIbSnrFlZH0Sq9rU+i49r z-dN$j8@_q1(4=2ET1auc2*jv|u5!?#4?`eD8eKZw)U)_Ppfa&9T4z5ZNX3b%ZI3}U zF#_YJqQMnh)1~{7e7*%#4FA>)hv%Zk94ICwqoX@68h^0PBiKW_^s`x2+w@npX5JJ! zNy91NN~_N;^(p*;L1t18HKWDkHv=k#a7HP_qSFK~;eEXMfNf3sy5uc(_M0o8re)L6 zo_6z{3q6fiOT@C5=X#*S2epWBj0bx(h);>tZIl!$1K^34l3b3klW|{JCSBtMy{HNw z@Hcz%TL+avIZWlDn+nF1;EjBApY7}}+zM~ZJ`WD$-46-mdTuu|QMCTz_mcYI{3k2* zFY%*=W}Z3OCR)JPxj4ujx*!cNAIvc2oj^+9#sZm)eTfjuGECG}!K zN@u^%RL9PgkbO{4bg##I)lbcodx2ZH{e$+t6Fg-a74w~sc`~Jsz8*D|ukSy>o-Ky; zr?>$X$7@9;Z=inoWu!3)3N!q>Uj|^zi^Gbc4n zy_PN3ftpHXv}yx`p~8^VqP%K=Gq(Y3FjXpgtMM8q2{=JL{pS;>?slF%7ABbfH8lmu zI8m#{tzwCDm*Nm>81EaCL}p!~>5f0CW;pvW!ugI%s`j~dvUXoj(LLk49|YzuZ`|wH z;%U7_=ggOhE;ErQFyA;XvmDU~`(E>rnAdc0?4PlPe*S6b4k~(ZP02~b=4|g=FGe=; z8ua%SzIzUZpz-rKYZLj+U7O$KqeoOb;@nzz+HY$w{Kau%4JRHPTbL~6+(>+utx$`4 z6k0PyyQ4$oBZ|C_AV3~|VT^*Z&@g>Yti%S(8nm>3= zE4(%3FM~(K>uei-1gS?nu|969FMq8h&bQ!oL|F=pgffXGN8h?FY$-ji%PZ8Q#UEo6F6u zs=b>#@k*WCL0*BIUPsSK+chKbaZ||}G20L4BgzPf!Wn`03h#sy7Wc7N*UB!f53 zr{*m-zUFP`;X1W=$Pmp?t>E7Df&&i~q0ov>(&yAF!iRRZg1_0Z8gGOl@+Sz!X8TVT zCeu2kh{{&km?lfKF8l1Y6v;IEeoVFKWpJ3N!K+aQJuqsQtL+X)-)#=?)`G3}k_P=m zL0zlU%6-z?DM8NKFRDhnD&O8^uWBHijl}3~!j7QioT3R9g{httR9JvPS=U2W2e3CM z`|rbJp{!e1eEWv+iX%Gm7{{#s>m65}C5<|?1UqY2-2Ii0P9V|)!pvHuwA&swZ54C)V#w-twQnoJW0O<=S&m0} z%43%R1c#mBDE&|e{tihYQ-PA-&IY73(uh*fbW0Gdd2EH7MT}UILbK70>c}*~@ZvY6 z7PxY&ihzn-ls3NwKWLRS^lD`h2sC3TRjg z0{`9mkN<8F#RYNK;KQIPe~Je8`YUMVuuLy0q;vsn_KkL`>%L@hKrpASDKS?1`bslJ zS%LPE`~KC6+f4WzsHN|DHFD}!oF5rd1*)xdUgCeAlCUiU`$gW8arIE4*=f$Cerg^F_s0rdPACCCv=hY8tlTKGb|0{A4~nu62AO zO>n}@{KK@m(wCfk-~NP`4(bPNv%q$Q_E5d6rATTGS);ARA0mQg(L~$s+3Yju8+SI9 zP&uyRvmOKW#qwx%1pFgmD`^DORJ0ABRC;3#1q?Gq$H18vLq@;4= zJ3Je>NF*8|TMgXrY&agwSI)BwwH_1GO&`7(fB6YxM~uk!|9QN;9dm&5abD5-)#$VE&cMGF4EF z6(M6}*rCgI&s@qNjOXr~8X&nnts9@eQi5;ZcP zz8=^M3*r=OE=58SDcqS2#T~jt`hrT4MB$}~0g4m|?%=q<%J&C0;bM3yR_O@`JyV&O z>|&eQO>6GVC%AoM(7yAJsA|@gIHZiV)xDXI;r1=+GkHjiM~*3K=Lgkb zMY=&`W0|knHzKXNtLupL$1XQ4JWUesz~65$^^=BJ&0}x1R%kJQVh@kqH7M@5JKt!` zz0S=k0yV`6WuW;1ItDH44gL;gOrq2J_ZJ^?Di)bql_-(d!KOs~_8h)G(!JjmXiALk zk$qmpvsg=Ayx#E4m^tZ*R|}UpSOmhHMXqh#4t0QEWujvfCq<6>N~VU`Gp39?z)16eg#JofRg z|77|%)7jaO>ZZGS23#!U^rOB>*D`S)%^{3=>k*#KnhE$0fPoDc^;C;9Lq^&B@l~tS zYb+0ZZ1)_Kk-N4>srJasHFku&gvg=UVo|$OcPoI=5wkU#7qXFoxC@y)4_tz$y}H^m zHD&@%T)2gh_P5m|zD#0rC@6nbe6`v)N#SIAC7z|9hEFJjcWW#riMo>e%~dMt3ZGvS zxY{Phxh-qUtj=e`tSc>wyW&;IBEz+^N~csy`F(bc^Tm#q4oypAHQIhtkvpGb+G^+2Vm_U14C{ZQ68E}{ zj(=9oFsM?SC{#EUI*klWA*~n5k*<>6Cwz%H+bQJAUf^}~#J(6*_!8~UpV6kcIHC6K zaH4dy#v9WLE<|EPrrxhmbJYZwMD$3{3m{!_q$%jpc3!^Vn(|m<)lpjDwgq?smmzz< zdu&vTV2~E`bRi})YN@1dB+lgi(I(}w%bmX2VsIl2WF&xTJJXXIy~uLs#8Rrvp-2N$ z*}XY3uauUcAFYSeoGdSyTGZ#gDecwGG#yu(#AKPeLLjCzCSzBFCQE_WKiwNZ3zk)F z4qEaHSk|d%z_MemAc$k7-aU3sP0+deM|yt4D@ENwF^!bDX^%Z4pUQHTgwW`q37?9F zQ~~M6Xy8Jz!Hw=@r&1PKUnkC{#>AIHQqIpt3>o7{RLF-xm{;n4oz~^gvUz{l(;8c)YcwpMMvMhcvB^>>y3JU zbZ7-*AN7x7eb42zcaW=dGg-8S*-X-)DL=gZnC`k&jg6aYpG@?@G8Jel zO*^P*6f~DnCY$c3vWTJG?)B>>8AYrrDhktTwXZ(^TL{=sS97FiN)N3K47|J`lq^Y*-Y48@z2Kk(-)V+ z0#H{QMbNeKP#et&_M$#L&Wn46HDAgwkH0a{;f^~l*UV45>8q#&S7cywzPbFlP9IUU zv3MZ$sPVN=5LPDimQInKUb4C7iI=BwIt0;L)y}krmlCh*94P*^Gm0RkK?A2g@j%Nj z+8d|SsutFa8$6<)56@exn`z!$`S?aY;km%QWz<+Lx|pLDgr*G`7Yz38W(fp2k9aha zZiE@Wn@TBRB`1+~&gJqcGNCdI&W^q^Ua7Ahl@!(t?E83msb#&)Sr}#J#Q|<1Qg^kW*{i56cj!gJe zV>A$vj$!5Xx>7$Gltmv-VN1R}h;v)i6#nf^kmH*7v*Qitd8Hgmt;qNJ(4#%EuvBZ4 zb&5=H!dzflUr@_!xtdG%L9&Dgquw#wI+wP4SmYZH37cC%Q zhL#G9`lZxG>R`{qM43byQ;zn7dM4{Rl36d_+Ckmg8+hhqt+;_Q03ME;dpB6zmRoD? zTc``CZAQZ*`HF~KtE!06McTlvaSY5$)!iM{y-#n7*}b5>Mr zMAiIeU3-GdK#f&n`<^VuA5ugMfl3+7KK^B%1GX*9HgAxb9Zy_e6utcv>PBkrd%LTZ5$K?6nJc_}yBGLXBr6Oo8 z4-V~z`s+5Nwh{mgJx_lY^dB5U(BuQ>rV&?b{Tol+QQ2H9I-I>G<@FnuODKmoce36> z_+j;wowp!ne-kZ|ZesJEGyV7=9lks;sdSs`+DQkLy9>9b`g)`!%^)CDv~HnY;ya+O z3VP-f7qJ*&OL8cf9{cTt+WM91kgck3Qk8F1H`*bT0kTcfT1qs@SM z!z-CkWf3YKl^P{q_>VH4(ykr5*igOrIxPoSy@a?tCM)qFBkB+~&L8s_i;yh~f#tga zr!jgck%PY0Hsq})d#l!yPLroaXBkzBWW$@ZK$VHh#GUnmpdtEoRrSlFDeUoH?BvWW zMTP6TtE+vr%m0Y?2!pE=+5Rxe8xYZ99xA6}Rr9NM! zrc#(;$OmeI4tM;{}Y%JzA4wy6gz9E%6Z zg^JhF$_c;N?`%=c%xl6I@wdj~?gw$g%^PaIvk{plqJtbC18>9?Ak)3=sGV=kH%Ueo zrer;Xd}3nRZwwclKL)W?|yz5_Vv+)B%25qYs^y3xvGcmpz<`6`Di=*EDK+{J4u z4iia0r|d|-5RE{XSgPN1sM(50>1K0xtBI4JaJ$D8#WlqV-OXRw%_ zjuA=nP@wE+sNb_T)$FDH=Z?l(o1UM8HAjHsku8;b_q`tIMRJ`qzw;u7c5i6X=tk8r z;~5Mih%K4e81aLeU-kVo<%zJ@_o~t|pB_#}NiBvx^MU7vJ~2lnohP7s=?}Q3johwm zLC)@0%f4d&FxH>;?Q$QLL#n?&DFxKHYn!@L$!3WVIMa>k%)~%d@ZfhImm-pDxh+{) z?gQi&FT5Fd`NGy>XGINLh_HJ0;Y%A4mQRnc34H#_$(qYa79oYnW)!gor?dgbdz4+O z>h`b`^iQw$xixIY1+fVw4LWPy4F+cNqYXe|Ce@hwZX7XRmYY*N|0tb>NRCTGfD=nj z`iDZ9x=xHSlL1njuWETDD4f^VBe|yiIG4ZPzg!twah#QBKzFd#bR{3R%UqXVb;@al ze*4ON^mDS2UwikPL$15vSkbqPHHDbeDK7}ZS}8TQe7FD=Z!?Wp8bJWg=N3H;ALRyX zQTJ|l?Z50x3#F{|HFOpuh^n3@p|8U03hJS<%&xKpqDQ-!hQxxtGz#wAb)WM?@jKnN zx?X`}QOWLf{f)XZcq?Eeqqm&HThWVQLIt+`vPQwnV9Z8;oD>P*-Liw_Y{4?+aem#; z(Y}@=rD=}f#@^(#Y#gyO1jNQxh^y;MwN^&!fo&Y~(+7Y5oA<6pTgnr~-|I-Z9-8*WDXC8+x>t7w5r&@h2esZC%(vd7paoG$aF^Uj75;8qTEYW$ zWKQ9l@Rdyk1+A)MB7gbecwHT(%1wAJC|!;btRmE~WMzK!3mu_T;m}E?__T{dqO+f1DZ&-I=cFxi<3oSt5ecG(GwI ztor{rcXsxZTUET$H!*$Vx-Ea~iHK!Nu;p`xrB7oSbc0pa-n-n zUte!|pU!;^H%|SlB}=W}M!wp7YgO^)!?vb0O*{NkecAs_)Q<3-`Xe_O~FB`7iSCe?r}V7WV%#)LyF4>nNQY zJ&3JA$%?D%kt7c zgZV!tyZ)8+ndtwKwb*R_z5>AXYf0jAK1LGTH2+t%#N=;fAe#68wPNzG)cv2UYq9@f zy%wLVdxk_r#RYu_Y-qQ?UE6WKOj_HwY8f;9XDI)tq}R^{ng4E35%_&a`{3bsK<9VB zr-Me`=!AId?$X{Utd&0y`73AL$UeB%5IHCBDqTS!!??n zlkf~n*3FWS3bNyeT<=ZpwtmeTXg}r#=&^!W@;p8{k?a;e^CEV-7IoMQtGhHJ(C9dl z(7t zUX=l(`(^Zy$YvBoqAR8={w9AqgK%TX$0DaSDvO;>;)+G!`f7-5@ImN$yi^j|ST_nI z=`!JOb5&YcbL$-q71?Y6el;Gp<~uv}ta>he@k6a^pM_BP#@Hd4qtuF2lU(O`*+m#? zPRCcl*<3ZR9R@PII2KCAOZe}qt<5pJYoeOUIVi7*AFcLdRRVeQ zrmUQ*Ms(H&gQ(e^)yeLH=tXNps7+IL6V^o~3oWWMOE662D!jZaM-m}}w@!6T8_(ih z752NW8hOM2JQfZq1ts~?Ynl(Aku8;^pH~Y_huM%E#NBQ@GfgBLV0B%^Qt1Mw*rc## zJtx#k$?oN9vG>hey+IuA5TgdNxo9tNA`zaH{I*zDFWd_A%2ck-tPz}+cvyW!4SnKX zRGF*C^r-N+yj!{7MvZu}Wx;pB&yIUUiS^-UnhS3~d)5vgB->P7u3&K-XL|(cz1|o` zJ?KbIW7H^x?ye4Ydf3ld?M{x(c+bBq_yeAWba$oS9@1wD_pY$cD&fO z4HKjznoO3yG;X)L@Dw{X;)bA4Hz8boQPH2iD1CD1a;%$DG&-lJISt>|u5+v?Dyi`V z`w*9yPMO0o^ps~_R?4apRKX_1^di|OL9g&SnZeboRulpkG{M*86pexPgr*;-lhbT0 z{mcfyDb6z=G7*(lqpED!Yr`hGi7d2zcHy;eL(gkg^=7Rs@aAV0J*IqI<}-|Z%PP`6 z1e~FIyfNw|F~mIgdMlUC*{r+6`>A29D`X*}lYwpaMuOa3?*>+^ zH26Lp{3s#&SYED46;m{vzPRI;oiC?V{Qz(-&-b^!JX+gt$JTSwPD#gI#Y|CtA*MXw z=8XQlGdbrrb(6PsYV*0L@#yI`c~7?{8!0jQSE#Q(G+4H20h^rFA8=u5Ku& zPQkS8TzMmlWshJ@m9cA?!Im!Xg9|&rNpP;Qv&@ArU?EZ5K&r;kM*yiXasE)Ow_5*6 z=FLw?(8Zd6Uc;1hUCZ{g3fONL9C!`*0{9h)9Zz*@ml%iZx6_0fs#hAsIQQLZ%fm0O zmY3@2-4x*nx0#$^=XLIN@^~IJkj=u*)+L}82X7_UolnPVPKnunF)shSgdMv3gc|KzYMQ?WSmvq6z_~GCk zN+6r1)zL-EMbo{qGeP#^rfovncdMR-sqQKU5p#^_M-#%B)%iNOWZEb-?YWhKAc<#T z-9bU!?Jw!)Q!Nr%b9^A zR{~Q>IA70Os3Z91ucYPP)>Ep{HXEzD#I9VlA?sksHNar$Ana2#_sF5Fl-Ef??5SSe z#jvB6yt306c$OVTfW_3Eh^EBPg8nJsLT7s1Ti^5E-G^F(t5Pj1DzMr_i8F}TOZw;Z zQrJ5!XatzaZ*`F6dfP0?4k0?FO^bAN zT7thc%sQUfNeG?f=cGPb3B7#nYs&%|UF*m%2ZmkW088m4Q*+N#+G=X*MvCUwcGERa zcGqKG=L!w0kL5C5uYPOJ?b4N15KCh^i_x~Y`yemb$ASlZ8;C$ArO%L|_-}>nL7+() zh?Cev&I-8`VXIGokGr$`+#x>8WNF_3=?X)~tim(xW0o~$tuB`M!w*3*GSsn9t}&gaXMUE-78p?((k^4tv#i58 z@T&4L+zH3#97lT$EupN(%8x&GDCQ2nyPi@NH~*C_hJDFgZNvvRyR-GMaiXR~ETODc z?W9H@j7B{5DZV{9hCF{%BIXtHHs%g_pHZ42UI?3iu3%?X|EOIJH zrU+pWDQ`4-IL{47zkm&}?-d&aEN$J4%Hb4LYPp1~%&M9PkOwB= z^knr-^)Hek)NvGxIcxQHZ-Z?*3-!LtSst}T+*9Nogy-po3aih@R|-QMiMdqpu?%`S z*Q2{I!uO++D8s_ScIa9NaPJ7h^jwctbM^j(@GY(3drcp`iEYp$rD=_?3)bb}41)zks=-LB?Tx zkHzY7II{FyzjZ^IrgGaN*7jiAFp1hwJXpvdpQ|LIW9qf9RSAP#76zl(=U|yft-3{2 z!Px7mk!2K*4|WA|`>*ieV|Y>2)qZGy-k8AJoF1vwil&KA3m{tf$Ea|I4N@>Y%=PqeYgbV8F=9S5~K${(Z$2dB-q&0DZ%`=s&S;}Zw$)xg<+Os^Y z2%d{?=a)5gr5Ay(qi9l6d`KTLug2Zuo1$usQ)q${FQ496 z!F?VOa$WZ)ac55?%< z3Dhlko(Li5p&2nA`SxHnZN5%5V{=6)rD~br?Q&ab9}Ck%C{0WxB9ZzM&0UVfz9*s@ z<>6YoTqiDE-Q75%3-hn-%^SK zcCzBdzQ;{4zP50BWZ57;vJ7<9FNzYPgjZC`ykdTV;=6Pa`?DAU9rltlv2DQsu>lJZZI3d}XJ^5;T zV!~J+^S-v;mpX9SUYaD z4}I})4B)3P^cgtN6ucgwQamvHwF3iUi|t?|KeBzAfDibrnR74b!Nmrb&W z89bN^i0+0_+|VRCgTUmD(L)KFWCoK$ok#h)R$Z;??MdLorEoQuy*Mei7MBI+4h6AYx&Otkr)W=( zY*|6)E`ItOU|MMZ=1_DnoSj#jRYprgOz7yklnK%8&K^AE3pfWOp5#x0@(Y3Q^0TgS zq0HbiiHe?ghV$#+M5uFKX(g+IBxZ2`b+Pz4jSuj1Os6N+jE;djPiIu7z)@_mrL2%h zG#3c(1r$njMOflm4z(VU>pL1>D%D5Nj3ex3i&$O{B&X@Stqq#vzOcn=+W^B;#*Su|| zEf--YhTv=*$1r6)s2Iyq$yC9bigv4zXvGF|LDZhP?Ph!|2tI4+w}9e`lpiZ8E6wS_ zz!x(tbhUMcs7QN_YYS6j5}&ROJYydNKIxo1nUHc%-BCgoh1yn}b|{-^pE}kqNfWJ~ z-%E^;vKsznTQnBL0#OeO!?FsdS{-hgUO(kJTri$SM{Bn1E`v4VDU@e3l%<_By{qC< z1YX6zu8>GoNC)#_ZEMql_qReGn8BdLOHSmC)`?4}o~XwrkRsToK3*rsj)3=wmMDP#P;s%La@Vyw?QgWZ$XczE1&*AqtTU!!D3bjXU_MVPV8O3 z01QEaAj6ZC0oig~$odyUp9hRR^Che1r@3R3_*7^lI6glbHVW5unDd=SOG@7OIFgUB z!8@AR?&Q89CSWifsPd=Sx3TK9T~)IiuUJ{4<~1G4YC;aca8wk`3KsmrotNi9+IVA2 ztK)e~tKr9Y8846z@`Fm&4YNvlzSJShbJ8o}2xRIL6jLq}BXE)}}lUOYqh&lxJzN7TzZy735vFTa6 zz$5Cup|thSX^EkzE@d4)Q z>4C@N8DA3@{eD72bZI{0&CfDa+3KFrA=xp~Qw<{DK|pT0aKMzX$|(l`0V;mmYvljs zwf^@2i%tX~=MYx~;^5X9Du0m3#_e)}GFywZOoYXJTW$^i4ba@2@coo%f2jI&{nH*| ze0tnjl_O^J|3;j@5$A8j`HkfLO>zDmf1ou2oZ8&WwtWn^nf*xa2M_!C=`Y{)FDw=> zg1-SMNauLvEH9-G?~04rwWYz$I-}R_sBmBgl>|Pd;&qVRV8d~{ z1(m*cZ%B)h^5a}S@Sd!C*m$Zp8+$1E9!nkb{B}!Z5F6~k!20SrzSjP3fcWcjh+Kd> zMG{4NhLGn+xz|Ed3iaY|TAoK+4vb9hosU1d^ir!KOqZBp6ympne>1_>V~anSOR)h3 zHy8BTF|Ez7KOBA0xiTLVT+dlE8!qPP3!p*%2AaQk7rL-m+TTokeSk61Wjs7eu&~fU znI)-g=+OHesy!Fa;H@mE%Ru!Obh#AL^fX4MRC_rdQOLtd`RVn9%p`8=?VrKL(D(T6 zG0XocTzF+(my}KDn(`=>o~lNDGVw&y)(NQ$@njKsGK6T}<7eh|8*-r#AE@3Drp!qy za=T5;rR-Voj=BiYwFx)m9Cs@fm?&h#lxstabZ=OcYiB`FcRpnzQSA6fC9(0jcqMei zGjn2aN6E1jcYDVh&DdmJZx-t^-IDPesKMELmI{2;NMvko#SKvE2NBUiHK4VkqIL>H z7EjuvZ-Ai9nUwi{-+w4{{-Ux20O%el9X3I)T9});t|Sc?$>tSJ2RQPCi$az+AKzBC z7yS@hZi<4-r2VSrtN2-eTkX}8)HJcVGsY?&3IHwlt zqJ&2^2=|Kw5}QdB363Dqyhw@b(F1!rGraQAvFS^A(1?aI{0$~C<#X0?vp!`#z|vDD z-EmiW+%6wx9IsB`>D7%0*&z?rL1dxp>g0SObI~r^m%;3f91^v$Q4FEa56a&8qf*Ck z12{1yqb~r!i+cb-moGq8XS*;AaAw=)^GCp+wrT$Fw$PtcJY3Q=ziG0hu>UkA9*Z6G z$W1dVTgol_l;#cz${k(H+5}bTfCi1PfLELvj^$21jeWJGF?-ps1JxMm#pU{q>BEFX zo8GN&=MyoiZPrJpaE_sDP#f=afn&fFGjFB_PpoDE{C4Vpj$$F0RoGg_Ly%yW*FN__je;qw#p6MJ%Y>?cRl6$@9ZCQlBTnZ*o~lw<%>p zY2!5P4P2#~H_#nqTmvCkHJ9@skDeR@{Jn7Y)^dfZ-~~O$Vo!4M7Nj z*_QsBQYU@>)HoTUkrQ?MSHS(@il#LFQDOG$3gxb-g`bM+N!+B5 zyutC01rC8kPR=S?-X?3n4)=K6GY=Y5lbVOKU+O7?hkAFsHQ@}mQ-vwV>Mj}P32RTe z2>0Z~2etR=(kC6C{@*}6`ez^>fktD|Fd3k-C2*R)nacHCVXAo}QuoVGV0Ta%%7X$KnIy7$(X5S8;lh();%O zG*`|ye?+}-iKQGBm&Buz_0*Qwl07pE-x-@wbrCNo*N^Pw`PQoKi4hGpzt>y)`GEd# za=K5(v?rN@^@m^mZip8-d2&)ACG7wlL>D2#`-YttFTZeHRIiFCPmD_8aWlmQhUJU$ASy9R z4GbSKc-97WPVEMn?ecD`tM-z}Ovb}+fc~=B9eZA5@zYC)oW!7@!8($xyDe<-DYcItSM|CTG@Lv*9k5`LF(xUICpJkQU~n9h?f%lZ zU-(qiM}nm1tv0BQR>+@DabD%6?c$f0H(-UbTz3B)*Y|xYs$29-WTm-{q%JvSUo-#2 z$6npn{JD5f#>Kyv@HZ6x#)V&_PX30%-%$8x1k%62g;PJ%5{7Fg!6Jt-D+6Yk8aTFBLMeZO$-d zI-QoIw3A+yNAC*qFtFBMiGvdy=^5!CU8D`oirx{@JI$);b)hxKQQh%|2AltRpRf)K zY=+=U-#d}sP+ON#MyE0LX9_Llk}G6YD>+h`ak}{pqyHbLQL*`Dk?2?L(US)Jg@IbR zNb$>P#DF1!VeoO%LTnTT+|ufV!?a79#RkKj_+e$Fv7S!(ikW!Xh{&^zj+*@%s9q3& zEaBQeflP06bLNm75s2tcGVi@se{V;HKzv7VPYVaabx=ZKeM z`xuCCuk3e-T%jFZek1<5BylZ3SFh+Bz}$&Fe%Q02Q~~gxbTSh*Kv;DhiryyO`T>&- zL`2OQI}h%N1q)sH`mV_TtMha{{06u=>SQZrXWqr(d$>NpT)kX0XaJdjCf!Gah~fSs z#3m&7V`E_g=+J$spy&mf8*DG#P`4P%dGl;qqu)YP5-7}KZx#siJtCXJfkJ` z4Uj`*c|_-+ci#Z}H&5g&cUnKxQoQ1e3qg<>rxj?rt`zm07o z7{wmX@G71SQ?i`mRIA z6o-RMSFAM2r-(?!`6?!>gqeMq<~o&wM+3H~CU>4j6BKBaYlTaEJ%KIHr_p0KLUiYe zUCqkUZc%b_Yc!9|vJE@fdlsoRZR%tvxRe&%5KGr;tIWcYM=Z#|%Tb*b3T#N1lzuCV zyq=3y$Ep~n@IDxSGIJl@d#yWf;N_sM-KLj2oyiLc(N(gP+lM7^(Aap2UWGJ2L(J*i zLCaA`$gE_15+rd>2eE{WqGyQIlFfpSw=rrdd&ddnmQ7M3?xNnz2Ep2JqPa<|tlEl; zY9ja1lDQrVgT}<#;oIdy$Ni9o&uaEZ9Z`w1I53^^E>DVTDJFlOLv+)-x8+kAt$ACy zls^0C(j7?}(`*NA{v{3W6kJ6=e@Cb&Rmdx_YBF1~bD3y*&G-_OgkZyh;0IT7e-S_= zFW}+Yn^z|~U+Ql`5@@pJxEsy0*d~Mp>IT3zN2*SY_O0mdws;*7y9V`m=sp9%j;*DQ zbC7-x)K8wG4C5r>5!PEn6(z%=OOl6@@5gzf9N*_n87hL;7({r%T{$9VV&@zwVZtn& zoe*3dHB9E2ma*vm4!NDik=m+IYRbH+08vgW_nW$Q6212gkVJ8!50EFR6*^`o3PQrH zM@0Z!Anf9i$_c-Z)HAv=RE6{c!F%(#Rvgdap-tc-sZnnMsQzLDUaTqaYYj!g; z(cH}w`VjyihA^G#F(bx#79Pc4d3SH2?e6z=o)#I7T=RcBrse51c@03q^XpFBOoebD zOYz-x`sRwdoNBl`oPuskNQAm&a|1AauNQAz3%~KukQm_d;nwdfyAaQM9iL-nZ?}=E<}q%Qlbe$r zuyI)0pyBx_{c+$Ydy?mjKJQE1ZeFyh>D(fu7BhNueA;+cQC&8Yr9u+mY?ekYGw6jV~oa>$0t6CX53wS9gFgLq1t_7dY;3S zWnbO5P-wV6F_L#A!FF_L9N8J_kif6eQGt2zK!Y6!%%D^wd)=#>+cAaTSrW*;ia#!P z^~pEV7%%H;MDe`dQC#HWKwvPVI$6)hM5*Q&{WI?<$C#WfT3Z#RnMo&?F-MeEpXfX< zx(B%SyDH|hP{rEP@QK=mkGGu7Qh(6<{-FLL|MKHPW1?Sc^lRcNoJDjwJ}5Z!&U7)_ zQRXg7)^X`d;@`0jfZm6rZuZU#3b`ch(B_=QO>9P=S_Jco5<eU%(&E^rrmHQ#XPRvBW3(K3S((5-h* zI)u%ov(#+M{F39b^g9%ks@=U`uNt*{*}dn$i@Q^mYIak&qn4$$)IwRY6&`Kq5L-Fk zt7ttwplQ$59TZhlYe@{WiR6&$zVm@gAy1>k&NYW48kyrE*yescD5YZxSl#T$JUoDm zO5z5x2($L}D$rLyoEsFO@KGx{1s#)8eZm*+z%o>7KZ?B+b7ZG`y=Da~;eX8C;?~Sw z#g<0e>xWuUw3=V0ebC(82GnpKp&OL&1&|PJ!JqE{$S*GzE`R?Z&m?uENSczqH}-dW zwA)=Z1aeloAI>3W=}g1yFX-8SQm|#7CFZiTyhWb!(^n%E?NUqd6a{xDmdC=%NYD7m1nt77Yq#{2 z3|_w4ioUK!XYmBd7{N7M%Ekg_!$QIifc%(OHK1eq=Z(a% zHI~EtcQ?6RQ%cgabCEP`4fKxG@aOw}?VJgZQ73)cnUcjeRA;B67lk zm*q#l0RV#gu z!8|2d=s8_=^}W;fq&`4mzKQN2B9r!IKKnBdF?cerpg%3QVRPW#*bX?==k_bU|Jtll ze&yC1Om_4fdjI>g*|@R)CaYK4O)pS`Qu0!DIhv&qUg$8(x7dT9N1 z_9@4uIG%#Jj^_k*Wu0a{>?Sm#L$a)VTs0`#%Q73^@)`IIFv)!Uq~S2gZJkeTV4@+S zq#|%%(^m{?^J?x+>bh)`hE2-{0jAHFAC2kl5<^at)7LH!C8U4tg=smx*CUSR@OsjF z80MZmp6|03Bm3s_ajOhCS6{uR+&E394*+OSI+>yN78s*~N#9q`8(f(zvWB^1!QO?s8G`g8EEdXU7%~2l zDV^2NG$WZdctv6pI7s5cKsrG9jy*9MQT$#Q;*;^!Po9ir`BlpRz@YopUytx#?V3DH zIii5SViitI;dnNV_`Q##;<0zk-FNCgByOMdUpga6&@+a#`5(xNdu6dZS*7ad3osA0 zTMi(%f@5bp1o`mU(bK_3)ZEI`v2|O*EUdaj;$|G%^>mgw__6~xY^b6>e%NX@??;pw zGstu`N<5~Sx-(oBH$ESbkgrR(G-R;35hKi@V=0*pne>{|oN_RBr>*;ScM8wWEp_#J zT9xHiJ=b-|{PR#(@EuB!c#vL>Ruy_NC%4vYn8C%ltHxq%a1sC@zAF|1Rs$DC5QX7h`VY^qSv%dKUx9SI&~9_s2@NV6JT77GDF)XOrjc#QGbl3<`ZRGBebg~z8atMPiBCFTf4P%$ zmNIrsJi4ztSS2&hh<5l?L1(S5w`EvZ{$?Rl6i1*s3xKui0Q@#qiH8n1^c z)3ovoy!Z{WhTWzQ_t!p0*hQpCA0Oe!0jG0_9w9`$`I&P^{46iAs_4=F5G+r%%pT#w zp0-Hdc*Hr`Z+&&?N~(2FyFv)(b+&Z|H3uF!A)8?y9FpJ2DSg*F-#k|D^RTr}zGmq| zq;e#GD{SlL9CZ#`ZI7QF8YPx*UW81JwTzJw?VHJHcg@h^yPVwmd7mi?JU`%8L`U~a z_wDk^l;m8GXFkT0(<2&Uf_LIXoSQLaSvJAJpTlpCDL>Jg7z+Fb;3M7)HwC(E<%U=Pde>*I3U5wc zHY-X0dRTq?H_Av_QI^hIjvmvevq&)4`8+|a4Jn^nQWZlzL%m>eV|8Hzx}=~A9ajro z`ZXxaD#s6UC~rKxd#{$Fbh|4#O(W~RU-Hh&LL2d`iR*#VjQy93XnJ$`Tj#5sCDSjg<~n(V5;dT4IqNIwb&m*5=bv;BMCLl*r+p+tB(R+m>B0bL4=sV*R zMPRV=6X*~NAEud(yGX~JI=%z@C7rD@egJ)4w7M+8Fk)><2|cOT+NE>Id4cdXr>iE; zPH%M|3=gW}+8&<9SSUplN$1UGtNRr)$@fhU)kd>SEwOTUsh#T%2UiuP#(AsN2zX>g z>s){fWzh&#QV)EPJ*%7o#rG#qXonb7q8NfX(RNWJjx#9rhfsAT5eC5ng*%Vv)f#tc z$x91Yv|I^s`Ji2_X^)0x(8{&#gsanL0#aIc6ccc`Eg+S0Ayf=)h{AiMLd1J|_2Im< z`Bf6ansr>|yd4SmbMFG-ED_p>6kFBgO}*^cWot-ZZ%!e)!~8OtK)On%ACFQ1rY1V&AkWKujpA|7yCOI41~p6?XRh3- zd-vn<{V2e6q5r&rAA)Bi;pVgxg->;j!l;ACFFzh%z5%rKXOWg(9M_!@kZYbNdegeN z2ELizJM+`?g~z8tvlZzrDa!I&jeLT;l}?scuRLj4yYSL{v8-0$`Uupxh?8N^2a`6n6i4^sFncqp*$2VeTV`)|L0==PH|@+ZRj zPow_w9qz}Nzs^B^i~#`taIOBs6bzDY%74g8KaKfm6Mwh@(J!m)$&aq&*2}Ji{l=Yt z8Up~xI{5-V)+ZnA0?v#z$o%17-~TurUcS+4Tv0f#d*7S619U#Z%wmFxsH`dgG9KF?*eEoap*=ZuR$nUtI_=UOCR( z4bh)fYCILAodfO_A20jQyQps=@-VpuMK304 z{@*qU#qiXfuJWB4(Cg<+eNi9V)fJ|{dPDA{igz#)YNHm5epP%NDb1dW?)Td3G+3Hi zuKNEsJpRSWWFu-F5)jL-areH+WU{`0l^EFaEHEgh;G zar`%%{T~jGe{nL(gy%B?gyB`5Up{)stGr|ux)SH^fz94-QzfUGs4mOhpp^Of zSXs|y-7SvAd>yLO77-n62uc4zeiVL?6S|A%H8hh{qzUkU(?tYVUMTeyz4il7CBIWP zBH=T~FUI`0O`>A>7obD;>b#oL-Un{0e2=2pf3w;D;qdtLlL;m9c^zTAdLwf=xwzU-L5Q^;M}0ZKO% z8rRMTktoL|hjS6d8VZ}791{m|-f-kMKz?vvu2iPbq`wJ|J@YdZal*c@BG046c2 z!N{Fyox;{3He4=b@1CKa>+r5_lN+RfxAA@ZovF7^(HedIZhf7X#q?FN;t$;_uG@#^ zJ|csIdh>$94gzf+&)aC-!t5(v+qq==cJ;l$Q~?K)fJl;T(x`T%sKmk>2`)VHan;%U zE+F+X8oEVFjw_bIR#Z;N7DqxG-0mAd>r*62+s7}m-W;Fla3eMKJaySiJL-?3N{3?X z`3Pt$DbVDXOWH%<02tM{kMxZGi!qMOm6ng*MfRLgeu6;1Eplqwu_f3l7ix1+i>Yz1 z49$pl=KupJ4`QtqLN%9?94aV1CSg#myl066hq)G{UUug0GQudW+==b>9qT9ze`Xce zlWj`EH2Y;JP&tlmoBd`+&+REYRQk32d9u`%Mjbs7no^iT`L+AjvGs?mge{2DJU%y9b zil>BW#Gh8$Jb}VKDLNXlh|%Dgnq*6g*Bxy<9&e@X+>NUVDR;-hbp|vvsx3P7>pG?2 z5DcaIgexU0cO!@v&OhL!ODV?6@64UgOU`tvJB7>Cruq#|lmGmOntcP0nzxgck3O|~ zJdMi2@@YZD&yvXW|%?G!OmA%`3*M=)&v9 z)o+5BUZ!6E_oyH}N_Q}gP}#|sIEp=2sb@|ETw~V^- z9m)-%gLCY4$6Eq*lcd<$-H4OY6hWdQG~vQN?@Y^^YN*Bq0UCv8$aL`{`z!W7s`gtX zAD6e*p5Bql5MN6MELf~21)G^ll&a|*rYoI~Gqii@sG=d7oMGFp?MDd7Yjv@9K#HQW zeV)c=v7>;X7<7U|f|Er`K*969AvY>aH{P--XMlU#T!O^{YL1G2Xv#xfhHA;z2D>R5 zqQT3VS0FKrXH7O8GBrvykPd5AzzI z_aPa>q{odA%5fCjk|<(jb&7j`y}!BcdrU&N0nSxoYwiB0KgDF(-iJ)j0e04H8?v z?(nl|WRhIH5T1O?aoh)ci|#sKo4rEQHRMB^OuV%VXmE}t3-2ELEL8`1WkGq5%$7dW zX2>$(VysmrIm}$ab4t?wsUl5n`)vcry?N}Sxu<-t~!w1s_Ubrxpzt!;-u(il$~@=g z7gTmxGMcO+^lC`Ovdm`60T7D_F)1asulGo${x*M`&nz#2Tibz;LGy8vjluJS59xUi zhYKka1QZtwxxQ{9>(~Rykjz=~j0oklO@rcj)_xC`;gTV32Az+V7ir4j)KVcX!tuv0 zqY=9715uTgJ?eKN_=}#fO(z>fMI~WtEil;IHSf$i)on_M`TqLoP)^*B+7w&Fcr@e# zZ3bN+Vop3jqxw_<$Jw5AyyOLs6<*uez4F?7}smbsi4SwUa#`m)=_$iT^}^U(Ray(MAu zM&3g#RdyOqCp`hz`AcnMY?cN4O?ivU6H^9BOD3vhFV1{!(fr!2FIEMg=G?l`%V2TF z+Thx_#-LU#7$XMa;f}u9zjG`2l4pwZE4U~n7J#k!RB<-*~TG zKFeWaEg#20>VJEwPZb(O5W9urAZE18K?8Tl+daR$UDz+DJr$?&cA1X7^MQyGesZSY zJ=I;qy@CYo2yA1PwDijaJ%6>-*PjW}MBgWtPNOqh?u;{zbSK;{UJ2Im4upiw#Lp}E z&PbdqC#)<5Jo0Mch9o;^RLdM3R7$Xzt1W$O`l^Ta z)MwM%s7cTXJppRD3F66Pn4R0Xqvvm?Yw@30*|JDsS~UjM_3jrRBD2 z#W6^xgYa2S+EAQfM1IdQOwC4&olR*jYv%BM>Wwm2+t-#|yAznPgrP%&*V+qMO}*5C zPhDO54an(a~Uhumi7(2I?0;n;~Wi7sEcZksui;x;m-z6kXY%s61dvZ&;aOF+6)`# zy{UIX*$)I$HAJk`eWy(8ix?slir&4R1oMg|unp$2Snw$N-=1xrrJg zx2J;I8}z^!4vxCIYkRpTA!SA>b%-_>%#WE^Vw1{Bo;8TA7@@b;j!=BMt>@X z`eJn9^oU_10KDtn;c9$(6Zs**|M-=o40H;TmWkOcqMnCoCRy9x?n0D*pwld389*hZ zhD8=QyYX7(U*MSy^N6)`!N=&g4$P{fSa^xq9jVtz_so%faOi__Lx;0*oJz>Z;%>ga zQ&4l6i~_!tn76wIRWx7?8^#9y14D;NqJ`HUbru#Ka~}n->OFtEBZT2(Zn zk?XB(;uW-+oOeHt5;UWpt8T9iSGOVv8wzA=_XFjGl73CfKJI%-ws%aR>$Qna%I24b z5^|M%!%pooF%#X!R2=spKiNDzqMR_jIMfW)DcOpO+KH539X*MAI>%VdN7yIu8;BWp zf5a3rjO90mQw#>v3yle@MO`;R`YjKaEBku`qsCA9NbX^S?>V2K zr*YW0U7j{}&3=Q-z90-|nMNYX7O5@U4bZ;5tK0Ub_8LOZ zj`fS4N=hvK)T_2Q-I#6%di^E$89LGVV17D$nvu*E6UT$mXV)H))S$h@+eI1ezPhT9 z50xqvqC&K{GPDwICu@TsMq+E#F!nF8Wl@FLc%(ZUKwBc9OWXpd|Jqnot#i9nz{Wdd zvN~i33$YWeip5*QBH2=5t;RsEaEp9*l(JVnUz8|%xes`x#bn@BEFx#bM_=YWAzIlyXLf;-e3CD9! zZ3}^gPr|txh0}tZ^9;8Dkc!>Tc*Od%;bIVC$J-!AAa7NviXOYn9ngXx%MN(?%D=ph zR^y~NsK2~!^78ZNYd4}cYR8-}6$YAAZ(ubiGO-Ay3@bdCW(1~@UfAp=D86i$c>xoR z;`9vb?tq3?0<>f6VmA=vznBs57J6BwrjsgeZ}xGVNc<88u`PM@ISn+tQPo|EqG!-W zvjufMl&e;Rv29T?T6Pd8T1taD>TR}XbXUtfgF(72BiDt{q=Hd2aRMCwUiDy zzG3eNYi~Ko%HLAwzrd0=I2a|+;E9O|bq@pJ<~@n(xB8mkqRi!e1Q?`)hz-adlYw9jZH6H_JtVdMtAI%dWnv91c4p-=sxaYClp371<64grz4>mW zZS3CZbh|7_n7S%C4>VF5x_X$WP4V?2#5pYbKKI$_SUW*CwIO$UYo*tv@6KIeG^_Q} zDBW;D$=4Rc6)J<M5?nYE#=MNqAX+DYYk9$C>Hc_&kzyc~l@tq;VEe-) z4C?73J&%Pq!eB?XSGaR&{!<4BP2(hxGo*;=K+hu6p5esf0YCwQ04Z_w`I~)-6HXvc zp5%1|vs-REi)vWU`zaRACOiUDEP2VNV9SNu%87~;PQikg& zm$}$+5@ccA~H*n%E@&$y4t%c;Js^Z4hv4>6tdLt1hnF1uNJuT^tZ4KEQ*;Zq2W zygoQ93z#xeL0EWXUTEndt%czAhpto7nGM_rF2$;WN&zcacCE^0%Lkp*pHSSuf`Jzz V!a&;U1*LhakN>$GaOT_C{{mY2EO-C_ literal 0 HcmV?d00001 diff --git a/doc/img/sink_connector_mysql_architecture.jpg b/doc/img/sink_connector_mysql_architecture.jpg new file mode 100644 index 0000000000000000000000000000000000000000..1195926f8bb794e48a35d1f227002ad2bba4e1c3 GIT binary patch literal 43478 zcmd?R2V7H2w>KOsilQRDi3bEU2uKM?lcQ3kO9_F{j`SJ?LI+XL5h()FrE7oy2?vb9pH|;## zXdiok3&0j|9iR??0jvR{G$aAI0}uyDACUnM0jExWPv3t}(bDO&r@yCjXV0EFd;T0f z{rPj}&(mMLe3Aabr3>fJGcYn-x_spd;}!agSDCI}VWQzH-C|IU3=W3+FG; zj{Yap(eD7pb0-5&eL8i54{(z4#3{xTM|FT}-*xxosS~vHM;FkYIeY5d`O_y(($mTv zuK;M0X)UL_boK1nvvlWa*ZbtD(`U}oF`m21^b@nVjImD{={yT78#|u_Ncz5>R|4^_ zKG+EI#2x-3CO$E#q!icF%m2X8BeqCV%If8Z;u{YUpFV%#5D-*0__bcv)BCrOxSZ}~ znkeo+TJlFjXrt-O$x~-f(4C`|sxZ<<-pP}vFPuGd^5pq5-|KvD2;-TnKk1!iViuRY zukY>=QzRkvg$*wFeR z=-)+jkZqf>xDfIYfT2n+h_JkaEaysSq$>jV8G*mSnRE>l;;&Sx+r$usGqz;GEt6nc z(odSx8MCg*)u@$p9sR^D6m)@8=IasQ+5zPR_$2sv*|qFNR{0?~dQF@`*N8ujn@xoa?MeO&D^n9x)0JOY6Y(c=j0~53=D`KudOqt2LaNE^GEt zN+2cS4y+G@*cf)p;7R@afLXKz)XR7ti}wUI>nz!xzCf6nqVkuiQ1bDR4imdX1VmS% zFjfs!*%^>vHfb3I4)`TN+jGs)*Crbk$hyG$cphPey3zbloy&5Q$fHLX^{I_vXs2zwFY}&ttq4vHfa_=C(EkFEcXEkyf?& z6FB?C^J%@0;^Xr|<oWz*P^(n|UIw-^l(PL|5L=2& zaPfr7o@yW5AN}Uum2La?nHR^_v}fjHB6VV1q+*EO+I+CE?%ql{^|Vagmcvnl_?`8* zc2JqE$%GM))|Z*02p<1z!!33oMM7T#-HW52P&V!#_2={0B(bo7g`K*$fubsvl^pxk zJL#Cr+5YoOW59-<-%vM0J7Ic;ji1BP9aqV{OOKS^O+-16r6fQL0ebD85V(h!qh8Ru z5~8{8`CnA(Ps*5$R^3C)m;Xku#Nr}g_?Mdh&Dryi^P>+MBDa+qYkhu(|JwfQ8@8Oa zn7nQC8K&RUH=3X7(L5EAZ~6dh_%eYz;s{_oCA*}PGil9)xt8Y>{M*t8Yt?<;@lpJH zg$v)d^n^hMVHovxAesW<(Rm2bR^vkf1$gB9=)KEfnAn>GVcUDo6;}cD_7``xs-_El z@5K{TqEfHp>lvJf@UrnPBO0&V_S7%jAX69U5QldC7v8CcI%=gk&$nKGiGa!&SrdC3 zI}+#E{QF4!U>Rr_$vZm9Ya+mVKF+SgH}YGYp6|PsRQnH6#nGG^@gclk=2Dz z{n^`sVmuHQZ9Q#5&DsuZ+iutAr0>H|hxE`iuRXA3LuV0(vYr|QCz%FTSTi(`Rn_Bg z^;<$(*u!e=+$Sf%&1g>5fx~TJz?Ud6hvn6QGw;BJ~L{D?LAp;=jo2(VbryJ z(mQiVMWK21F=d+m&#HKNwyHoS+jHq-*FOovq%K5pj!8cvkw zS5w#80$qjdl3;TYRo9^>fa_}YN&{{hwdy{+KV4Q1G&l%ZN3SQjuU=t!tWm?bqPxk} zUD$2j*z}Od%X|cQQUg2?7!LgHjN#S7W0e`ZE?wA&i^7MX%~#CDizF3^L&6(iZqVSF zKBv?mh6_OMUJcl3iUK6Yc8S~nK3?p1X%gVJJnZ( zaH<*#N4-W*ra+pu#9TXQ=H0&iuKJkij2`g_K=J?QPRkcYp?J3GvCe=N6Mx=Zw@#qCYH8$K@9vV^4zA|QB2 zVmgVzI%BF{zs?8^=3E5(i$ic_bEymdlZ_3rjx4TAOt?=^n><>TBQ)l2elX5YejL^p ztFz=sI0Ep=`e+kawAtTi7l_Dhp;}l|_#Zej$Op$c%o}*r{^qIbXD`{#xRUmP2vZ!( zeH}hLSi0b>H{zq%G@1*g_wqQPkaP&i!}FjfFD@~>JqOszkm3ds&CieFY8kPmg7TL@ zd7-_!evDT>@k-R2NA%BE>PoeBrqz1R+-7L!Kr9akm!um}iCS%*T0L6IyV9iPalmBoKaM?rnPPN=vT$debJ3S?r5BE?2bY{fqT&Afy zb7_M}=#!4Qov-OiI^l2~Q9i1hcSs?7#}A?@oR(`sgAQ z*4j}x&ZfwcB!RBz1K&3+bFOM@$g+ZKtNu)~j%hyIT&zVQg7YSq?3omVpO`5-mOTmm zbS9InS&|dV$ytCEz5!x6h}JgMH5{;*fO}|iVkDt&Ocj+KhqIinboco!R1yqIuX)fEUtC)qEiRz72q%l+NLpXAg4PO?p!mkz2YZ7I9T zxgxhCJGp|G-#o&WkjPIGktjrJQbr9X=73kZqlT2~k~d-AFY`_J#urK0b!xI|1tGx% zu{n?Bo}Kn8!m*jSMjZii9(T7)!1bFSPswL#kMsliHa-lum8=RJ63Z#Dfou|~a8e_{~%ZfsBm&fJlMEUX;N zN87l{$2855Z@lqX6_c!i?fFj!0_jxXP6t>hxv=VCNN3$8YwKb)1pAa%;LF(D^#1TN zQc#NPzm6S za+Nm=&A%$f2joMrYc9T-0UL2INb+^>^LOFVJkYY1R7GKjSgZ;d-ppCV9$RUKuVG3C zLkbJdB_+@1Cu2YS!a+sf>;RUeT5zOO#D_hF^8;)WS6p7lSS75MqihGs>tTeMb?tuF zjwpM{Cs41DxmVV`A1}>+g~(3xN+xKbB)la=xdT;$(-NFll7-yY^f# zwK=-y+xBh|@TW(>U#usB%<;A4(R}Z;*AZ0b&!9q)I#G6> z#rmg6M6qINX+LMuCoYS*rV&!$8&%QvFtbn+cuLJXTC4F z(5&&w$nbCS2TesM+80@*g`+U@8z?-oB`6`q9O-jTf*yJx*8qIddaYS(rQIPn0Jdnw zCr)xG82SJW4UG_zCJX-V7g0ET-<>1g`JSh`11SK>Jj~D2h`qai$F^+mOl3Ei_(g#P zdP>3tu2|>9w5y66#Ch{A4OwP^K|-O!8MAeQc9ekD70z_=y5+#|*ybmN_UV*N6K7;B zQgIV(?60O(U0~=;n!!$~BCf^Q?DAxBU|LANZBnV1Yx<4HZKQ`HUL~gZ-Qu@5<>1Pw zH^QO&B6YGYN*^v&IKq<6k+r?%@OgGj((Q2ifW0{p=2bZ=v&iJD1dSn*oD=uuAPt@Ofp!Sv9k%2Ww_5wTO1 zuZX23pEN>WBiuv3i%OZJ-mRYlI=kNbB|pVfQGOIu#KLkjW72LhS4G((EXZyqU9CC9 z0U71k9_V9sO5@Hxtcp{{bP(cLY@n|bkB!TQE8cKX^SGT+V@haQ#Lds2f&)$fiG-Od z>e$KXdeie@Ikq}hpN4yv&9PuJ;Oqfmy?UHYCCT@ z6T_6Y8`xQW1PE-Nt;=Jff% zvD`}1=;yOOOdZNM5*Q%@a^;ORCh?$Ku zjUAh#z#Gu25uIlZIUz!tQlaTNJ~v>h2ozr>G-F3pzIjTvp?}3sI8n-@F>E+3JiivW z9zR<Uk>9__WC)f9!PNozs-h}&51GWdgZUDxJI`AYAKO!T|QP?;hD16bEhNm{OFB%sz(-ZD#rMY zvyZAgLPZDjc0*?=d;83u{%y`muaEE@J27>GhfWpVhSUXLAd_P<5j4xR6E_+FHW)FK zYK!>I;+jO0r?CWsTzFB{YfEjfCOOSW%k_c zmxR4Wlb0Fb{0vwMq%NGAV-XJW1h>42xfVGn{jhX5_xF)2 z2j?-}JXbz{tl)63srRq^@=hekQ#2NYbxzc1np&u^+h}f;yoTyq!*P|644`AJZ5s_w z+br+l=3RefDTr3ULt&RbT5b5SW6GvX_2wT9OvkGBY8?7mY*o(BrxEa}PlceP_0x<{ z_d1o`5Nf=^H>jU&t)3}nqp&V^HX3XoABS{pn7$a55u2L~W_%Ofw^-ej${%@eFMnw7 z19e>9@V0+N%iOTemQ^kyBtS|htc2z@0F{Ve+rmZKO2sM{Wr43@)milPt}J?+eiFO5 z(+~vCm%Wyjy%q4vM&`nA?!)sxAu0^(XvSTyz8dhpWQ>8=l+~KonkOtM*@rGrZW>b1 zcLa$0dKrc4)qpx_CG%H9?z4P)t9_a%oE2_Y6zx3w!NQy0tQr-LiuS)N*PrGtR+md( zc}dpi+e6vL!oJ(B=W2&MP?6*9(&Mu5A)PQD<*N;72AchP=ExUvpB` z@eUSD0i?YMXU_{51Xm=9niG|o6$_s=y{yK`4=pO9M$$|Mt3r#3&Ik+kVjV$Yss8gG zR&O1rWP)Gryp_W_@xT+_g4#5ohR#NyDC3dM8XFIwKS|k}EM`Mw%!_oQx&HWss_AC_ zc2himd?p&m9ADa*0d;af1Zky9Em||8g?FGM_P_vvNoUiljRo2~;(W-TXRar~SwTcE z^5-+W;bm7*$0xaG{wKK25}x)9&n=Cel3zj1fYl)kiXGU)E0VRG*yrG%|8_dZ{ko^> z3#E8}(`tQfZbPt%?=#8$y%_ywiwo3pF6n))iAA9ocd8qa0P?`$7owf`ayMcK-OK6g zw;e`*-4ED;pAlSgC>^CHNSjiNW5?YJBCYbW(XUD#O0t#p7d2r##D34C+p%^wo92ho zJjBI(-Xz^$6q=r0Em4$2*0`U~OH8J+n26v+q=kdDoSOH}bQhaA-Ei6{x;BGQF&BW;~<729HgK!e-|}2Ow^d7%bURjFrzFF zi$cI22!dm2NzH;rR3Xql8wF!oh^%t`hxgsGLlRTc!?+)R;gq;Lkp8F-Cl|x?_2Iix zV%5D!9_I+8XEIshYlZWdi)t5(RwD2^Im`_hsp$;8PT7vODfI3SZC-fI^`cFRS1cy<$x%cu5wQrc;P|pE?f9bBi{*R&X7jFNh z?D=Cu?Umv2QTUHhej0tu4-L#l_ojbt4A4n{=ZqE)G^1Rb})U< zFrBum{y^z8UE_+W%f=;iVpOwBNXU61PiVi!%$PH@(5NU90(2|k%vj~3X~q9yBjw-M z(?30~@{i44Xv*{NQ~rM_JqiAz>xr)Y?c!1c`U6I7~TY@6SGS{KMQJK@NZ^t$q{rKRZt#U+{VUlpynib#`J5pHGk+wEN>hnj)M z`zEHlL)K1m=I6Bxtn!c&k#5<-?s1Z4T8n$nPmW#rv!dJ?%&gd3s!y;)xhEgPTpWx2 z3X(H}_Sv}vcHV!U&GC*qQNQxNXD_7xZ_#!Cv$f~r@}NuKaQG+1svZ?hezdI>fzw;O zWuz|Mo5F$ zSLZ*M`(#HT$}H_(9{MJUya3t^s}1gY*3WqP(O&u}gGDUm^=#f!G~^s$qVN4*_+FwH z^JZ3JH99PNHxy|UL3wq#KA{$al+OClMPaZI%NAbA81@OW0kF9dTkhma%lW7d{=a z+5IANGkqCbse@0#?)`MC$S5xA(p4w}%#{=GxVOhO1CO>0LtfkxDTAZQJyIQDTqq0;)vW#=L z*W%2QIH}N|Lj~YB3#{O!U_~Q6)=nkG3FDRLeo_M z2~U5_wE+JxSue6WQ<-S1>F*D){BpmPSURL`9newTudGK`QkF@xA_LD-qP91lKEI^x z^w&%B6|wc?;fk%>yklhEpKi-M3|*veyANtPsEs~-Kc9ew{+W{9rE@rF>sw19>2BM* zA2{;OmU9Njx=Ts7!RY09fX9WWeQ$>YMPV8{p*FYVfVn;Epw6B*g>fcNg`=*>;={ z=j!zUC5{1uVe_P-ja}_Iv7ji_eros*yG^jly$oNawp!ouSBB>y*&~Rpfw{uwio%%J z$8b0;+-z1RuVzyYy$TbacybI|eGhLt#_%})9^Dq_A2Di;sulc3P`!2xe-CauCgA@l zVV7#>ROL}}mX5`?rDp1LXO@ltF5=GSDpj+tjRCBv#Ht+?6+N!LjDF_)YLJ9WK@3*S z1;4@bT)<4zN3PWO=T8bwxCEN@DPZ=e5W~M@|E>e4Ch+DrZ+oL!DSbJm>qfhNK$M+N zQPj;LY(Nr~Uk|c;l}m^<#l*cz&s1zZ`5Ir6p*1zZgay+JClPB0)S2;UKkihZa6*lS zF*?s-5e{M{zmoD|#<|}+KNf>8do>A~4Q8CnKdpTYzRW84EK`us(Rm~==tavtB z8=BUc*fq;4^c-n1I9}$+07xmuyz4LuZ+h3Q4)vj+%2A|IwC7?t z$y9)o0Gt&JfrxWXt-q~VvgxYCIw4!q3531|pVNH>8DA2690hBjvltEm2&QDj(Pyn{ zH>}FP-f~@kLvW>#u$Zkq6pp^YqTHT>E1rH+oUvb$HeO{x%yz~Z>b=P_Ftq_QrAx|} zvNjCpQmi-nudZetEy#c`Y%HatCVgmAE_maE#s z4^^DimX9FgiWT1(fp_9{~#4G;U((kW?Wtq8@W3M z{&ViH-2cd*1lN^&qSb!8$kc!~p-prC^I8p1+kSz>)KjA76HxaOwNCowRQBMS`g@1$ ztt9f_fDKgwylo<#uq_4K-nUN|hWQRca$-&9IBQ<})Rg~8J(XKZd~-1A18w`^v` z!7WP76XYUI!RzH*q+z+jt4$v!r*#s;iz0PW!gCn?1S5l{`3|MPDsu)i$lgHE?rYIF z{b8<|m$3un)LAGcrAam>FZDnMDhBjU$wT%!hhgj8&Fq;vc~y5J?vlaKLSU9F2-b;Z zJmlVB;%}nL1S$3^l2d-csk>Im8rY!2^o;0{`?vU+yewbI%uOlZrnB(!q7oxL>v2^= zgbQaMN7ff7D33@3H@bWAUnh-L(KS=$Fmlx%eR!0zCzyo|!@@97#b+m`$DyK^3EybU zY1zA9K?mv%fd9KXm?K_q}PK zFOcl=C%aDa#cr0+Uu*&q;_7XOO5IZjqAP(cB+F# zOD33F-A`OYrp`|6JFZ=pb0lPZmwreJrChTImeu_#@`spf@SY%r%*cCB4$zEM3LRkA zu(PIrh*9`M?w%1p+%FA9zIyQ$n81FEBFw#EL?s)N%eOB2u^wKl20;_n%1tZT_~p$0 z#rRg~3z>0R3Hs^6f5C=G#&`}#T7z2%tAcaEL6HUr8M@22T46^3v*_g`Kx}4EAJwJE zbQ*5~V_r8jY=g8btn#xr>;uuguSDt8R3#!^rWIBPcLc4(6=+e$wizDvn;_jyKc(NZ zicrBN&4*{?W>IO*MDlb{t_GC|A;=)^O+y>umz1g5?ue|kj4jCo#e*3C_~G@`h(Nk~ zfzoCv4n?gCX&4vK)gu6Lj=RQ^*i?h9ox|JCYbWR=E}baqd%8jk>%l#9UwJnjJq6Qg zyj&_R#BrC)C(AT=uS6b!bX6CNm%0QYs-1CF!*ShkyS^ z!5{6E{2FYZg%i4s_B=5_QW~5%0-XGVMdyoyjHuqUVoV#Iwc!!KK5n00o-=n(A@i%j zJ-SVQd3}$T@;f|#@4K4|PMHgi;o8nXTVq6{{W%Q}#@e4WF&ucsG=m zf`=XI9fs~~A}7EZmfqe)k{%VvCoeO}!@)~CN;Y}YYmbj%@4uUNx5Wu0#ZeQ$neM-B zIHKK1Pl96(n0NhX+Y2NP#-mvD|D1h%4;Vm7!DbaX!g>vK@J9eOo`>9r*in7aX{znF z9#{-th54;p852{g?efN`%u6jbi2Jz*UC9T#(K}{AL#@=w%OjcF!-3G~xkOO{z17`^ z&9nXG0THAQk9Yg>p3|Kxa$#MpM@vo4 z6xDUJLryroY_fCO>vIY^i=6o(p>T2(M`$GtS=Kp~-EE4M=Sb!E&}MD{u|f0Mn&SHm&5Xlm%b!tnwL2YHf&NBMGdrsi%H@cH-aVLDE`fJC(W&JRok z>bsI^g@;IFIF-v-0#R9RNQ==2QE)1IUeHR89z!i%cvYr-SEO^M2*J7LFtJBFkJ+ni z79#~GRWBPE^z__jd8{~>QG{bKC)b*qg%O%72Z!CRBU)~kmRGTAasfH;W69yozr@I8 zGDljJ)nF;8h@}rPE-^J=4u11;H``I`aG;z6&<-8KY=cv%5spKrLuv7+-sNPQ(Q?{i zQ;hUzFw5P%NZ-g*e$hM%7ey0)euqaFU9$p`khPajg2QIYNkj@$`$S4L^dsc8Or4hkpgx zfcgB;f(A)>qg{EBKStKiznpRf&tvmV=>Aea=6*J59(IcrVLF@dPrslsZh2dsQ^svJ zZ_@*GEpMcP7OC%J=$v5STEV%2`i4dW?NQcullP}2l1J0{9a9Zd2w=P6_NEp|q@5uD zvnDi@KNDhvo$S%sU}Nvy_fSY;MFot_@}5y^?1%So{Svr&CRbXcU<=96P#?5w z6_H?|rKQ%GZv7H(zM#k^{4V(&S!zw!vy?WO$|;?n_XGzk??B>SqfB)O|@r;KsuT+$p>*tL9?3 zsr6(~;>dgs)6$S$T-nHau3@5aZoo{Ii%YDaI`5-d_CC)nQd1;Ya9GR_!QRLkf9Z>` z)b3W)WT}3aAqjqkP~5jEV5Z)dAOc%PC$QORgr*~LOYVPhS-~zwW**`XM)%s>V@i$y z{lT;wmfypypt;P>Zi6C4=Bn|)lBv7iwqQS70^w{=AHln~o-0vCwQ%&A&77gpCh^(O zt^Mh;H^@wJ3{By^3Ul&POY-LP*{8*g7fTI212oce9`4ml%KaKK<)w?Rvfwxa@=UPG zFyTFgfNTf>3xPP(p*{1g@mA;JVSeEX8E#Bv7g=7wX;%OIwfN4k!%D2OuJP0Y{ux8f zcqXyx8p(gXhh?%b!ywUJKFEJ3*XEviWZe*g;=YaGE9?-0jhYX4P4&vF>IF7)yfdb5 zB*gd2`TK_g9R{XXYxtba!ZDkoY9RlpMk`4(pt!U07A&sB2Jwo-!{>4s7b9C4H#Qp7 zfwKqs*+vV+E`=GI&A4ubzOPW8JAcDPqLPTSz>9mEXSRh$4DAb9uk6*^45$zeJi=DN za)UgE3(nr2nZ&HL(2P~oQg0Av2VIZiS4NOYaSHm+)I)80+NXCqK3ePd!! zAo^0HXfjOOJ?bUVHY;V!62b>8ruThE+acr#>s0UYqYNi!�}hY zj~x*iIcvgy{YARQ^g4{v+CpeXQKCML1{Q+qKl+N0Ow6aa;E;eEfyA)xtOP+!ehmLh z=0Uc8#ktVbjX+?C%bI~W>C*KQ7|+a4y%&PIJD)Xpp*gebJ#P13i)|~XhqXE9JE0vD zID03aQGgA2mxv)litE?Os^Dl=*w(ASE_0g9TU+^{$ z+%g)VJltsCIf+E;R0Q;291+oKH$7u^xB6n42%4)L$JNu< ze>*-ZIc6bR71zz5v0(_?<7bxtr7HJ15{ZlweaM?d2UR%Gw0|fh^UHcVuV%N$`m;`S zQfex!rfgoiB;V)Fmf^}JKKmZdg*nqSBi9SDD~f7QzHpK(o}?4y%59v`;Fb}XvW4~A=x2!^|buPSW=6_{rhyyc9=T}!(hhm*-SU7tP=gF?m7oW=c z@VPS-RBO#=q>Yo9K38`9FZgp; zDTkH3V@Ci1ImY^o;6N1V;!Dz78;`PfyYwRco@SvNSIT41wLq8fNwB3EzObfN|EJLa zpWTVz*Zc8nLXj8cN^bTk;6FP12}Tv62C^+@Cpngjs+|2tL4t}*);P!~fQt0N23XkZ`3)l8-U%aK%0I!|;Hm$b&T79^?mt14i zf(o}wj*Q{hP*c!d^@KQDoS)GWqQjbfCT3tH4Z%!lqY(-{B`=F48eOb26&WKXt zV+$-uvI~0S!y4iIgb8c<4fUO&D2q~+8S;F9dhtTKb1h52q*U{2 znQq8Heqp7`YNaD)bC5U}tHKRSM1WO+z%2IOQCn&aL+%4FIYsMk7f(I^bB@z^eK*KO zAgT#vJZXc0&7oVOoy#a-Z?Y#QB?1*YTu~QG5Rq(?l)sidD1d_-Z(y9k<-i6+)2y_i zstm$2xevdlO4!^Bwy-}m++#Qb=)vX2-0(wJ2R*!2J6)@L3Rpf8Tk=RQF@;5jI*lF( zFE>7v8)5q1ydmavZ}qrnf4gIyg#3O&S)j$ZHUl!DtD!f?I?=*L4U{+S)-q~?9+Gjr z*&o1)nH-$DWmjTE#L~gnkVyX|NvGsk^>>rjP;`mj6Y~U!mx@j{f8gE6dKBN7Z>s%I zUzBwBncd2t4x{8Y9HwKP4suisg@*YZjACteOEAn^`<$bu*LdH_Wi$-3>^*L1L1kN; zl;9u5sF6np-t*)<)5NM{SH^GuPSvFCJe{=+T8LlB&Yu8JPRz<|2FN63QN?6rs`V!l z?;6kA#uQjHnoY}kt0DsC2rIq$oTCRfR^)9)yC7}{q&wb0z~48lBipTuoz6t|L(hqL zH>c_j#{J%?^gdU6Bu24%k8gv&T$!3{%+r~hpNoEllZtDU($yZ?9*J^}qXNBd$r8=t zOg-2=gq-hPc538R_k>HsKjf>+3&s`N*IZRw$S2L0yGhT_o0gmLzC>I7QbocH8Q5S@ z>!2r959VE&d*oDm5X}<)BeoU<3nSDfrHQg-u=h2N{iEI8ONHNdKv=FV(B_t`o+FhQM|PbH};D|LAm~8^IRM#0p`@qSJwxuwL>BJ);7y6h^iE} zQg~9^Au;_9;R(q9mG_rCl!v@11j2IW)7)WfdQq`}m5jY>FV@FZaV=GS2|g@5AA)n- z7*dAGiWHm8wbKK|S>C94jz`{0)|=%$0{p~QO6Mh`Gcsdc`1gt8&v(Jy=jMEa7ux28 zFg=gwvs+-WFhAaQN#Zi{2r#TFZ>utBQG0fBb?h{}OTXtqcuZY~DuTA{fR9fIx=~2c zzZAO#tratN`vU*?`PlN4e4B)`_qD_~QB-ZOyzJ~ki&%McBx>4l(ckY~T6}Lzvs5}v z#X?m~*vDg`4~3K`iR413p?t?!0qUn+@4Olouno1 zr>Jzbu1iA8BfyPR-I8H>-X@oop0AY&B4EX#S45nkW{7Ot+qTqtc|AS<0csCAy#TB% z|Mawh?%oHQ z4xNtx?d$%v{bjYghg9VB@P|wmT5-#l3y5@dCnfg?&?X7_x6|Tv;Y{k{iz9%D()wb4 z{t>|7+kS9WBK;Oc@16vWMw+$wR={FjCWpZhAeG?@q$>EX!ya$;)I#ZKT+rr(&+m?3 z?UHFufRFn>bq8A1vW23&JX@9O#XI9CJF84)6~sge>8urSd6Bj8G9R7%pLFjs{+c^I zJ!`ueu(UuJEaP?V^4Hz#i!93a0^J?Dg@!1s_Ptr{yxA!_<1v8ePCW4T>@>2IT=e=? z;Tu|CBFMN2&h^S%w0z1$@Wz0FK$BP!OIFdqR6>xAga==(LYb$QC>-m0IOC982Qq7vcS;q|}>Q zdxbU0s_&K)JnK5TWFUV~e^+|=>dgtbqk%$HfHg!mQpF#{*$W2iAim_V?UQfyq@I14 zH&B08F&5G-ZThP0Q zuaAq?3>Os`JQl~3M%pHATSj43p9~C?Cwzd}MAb>SQU2w>ta ze}L*&*UaVcEZT#|yT;ted%=$Gl#v_`KP;TfGN5u@68^k&>7D3(yVg4x<}U?o1wl77 zgFX*BhNmFz#R@4OI`S*S!wWo9@ub%Iq*3!ug}IGdgY_i&Z~8r;P3=BvsNU`NoP7Ug zdaY<{aBMcrIWC_}trBw}muB8B>?|!|=)lG=-#3jJ-<-#38O}_E%R1f4N}yWFg8lR? z@={S)vR8f|L2x}u7jDqD*sH*%HDkhqxIm|abJ}cGY_M;=O2GQXFd-9la)tNIvz1k| z^zQiogxGf(ma+sCHv0DAhFrmSYfw%X0@jB@KJ>;fW=C4TvW4dr3v*)JzR6d{p+zVp`2|Hyw!o=!Q5E4$g4l4ktF8&h!AABru36j3 zpsRK=6&}x4Z-vj-L}-_^R71E53i5-rB5|KAAIvx=LQqvYuhVIf!+4pP{9@3hM(4GZ zQAwk1R9@Ju^`rD)zdAZcl+*0Zd|PY*DxBVd-^4Zrl4ab$2g7PH-?&FPoaWY>)+nYj^zu9z(+s)L!1ZR()A<=ol!<76 zAm_?cD9|hfRQjN4up4d8s{o@wV}D8I*(%2KsG*5FBz}-hrduaK1{k@ylVu(7&G*kNWw0z95~lbB{~0=#2v@qamy-E zRGq@*_)|_Iod`dlGb2W|P{v{=k7R8+9Cf<1)u{AWRTw*K0o2q4w7fG?TosbOKf|43pMqn$rJ-$cXqxWv2F9Yy|j zj}&hHE7enMH^9h}$L*&4dft$17l=nL8LX_l=j7*wkWGZ>)#-4=(G2|1^42{dH#UL7 zoiDW*xyOecu5Q^pJR`4Ik1UZF9Lr)1|9u@w(S(|RedXiPQ0F`qN!y5QNa12zLVr;$lycQ4b#kL58iJ?WTu3#1SZLG7}q?YSW zt0jF=TtvNTja}jLf)iMY18Ze{$lkf;&j?8w>t(B0K`mg3x<1Lq12Y1c$b`#~0Om&| zzog&*bV0M3C^KuyEi6Q1;O6YDcg5NYt2f*mda0Ei7TLOGx$1#kT2^?bwx^M9jW%%5 z$Ce@48_IJw%Sy=HU@~|_*|M|_IBeu(UY7M+OLc6lp?wyo#yN+Sh*}vNVF6E{Jw0j5 zIz!dy@Yy2TjL0l4cT9Ft6p2xkhFObPAo*NWOI!7!&wiD%RYjq!!dB(fV6SZ`*d3DoU}bPn6yQSx9o+FZxUY;!}8@VeaEnR67 z)!^;Syr*Iuo*W;FKa;czZ2fVxed1E>;&Mfj+&f1A8R+`nDeNa}batvpAzrV!DgI+` zE?7cv58+kl2B*5DnSOz@L|gAr$#{Qb6s%aPu*{LYnTTYNxIbb2ueg?BBc zl5cJ&VvRk~XBtpF-3DfPH%AqE<6PU?QwWvlAy1#V(efb*6I@6Eh>Jou3wkUG2}rfqu#IZ2QC86_)TsL;JVOGJF zcbptt+h&+(=~~Kfx2|&*o-WZ&dSapyY)fV@X(4Sy8~0#Z8YNxJbZU~y zQsa3zdx3AZ7?WYq&e%lK4A88XoUH;|h;w`8ZN9lRRFf54_!5ZXOmQ5FiZZt38Y1k0 zQQ@yG@0tas$HkWHM(ENPTGry*OP=FCMKL~cahT~X2}M?*-b`7`@!v)9*?D9*EOrK% zSuHwJ7((%p`;go1Oxh-@eqdQGm$~Vnz&nP9x*p9}{U%~K2c)Mxup^a6fb#>V3Ij#< zY-5iA9+!7AyswlVrnhy{xo9mF9{RuawPobATTq<~o_zf~nI|VH*T&_2SkkRqv9TJ7 zxxy2#c-s1u+-xGvN%*DKHfq_F`Lx2n6IMcQYg?;yK;D{qWeD(Hr!8`vKs!B4`Bqi? zk|H%O^Aa1tGRbEf$c&5aCio@`-!M*mlu)Ru6vH{;gg~23j26`2&wVGF^p$xaD*c;L zfp@32<dS!FJX+!RGV%ujd$)gY&{%Ah)Vm$baIqda((~3d1sdF}Z2PDw10rRf|6Y@wDOV79Y} z-|bWHnC{i&Pn*u_AH=aR`dIW$Z{!5>JF>=3E%ur|uz|FN00qiS=ZobjD&3mqoV3?+ ztfS?VKG=wP9u)bw))85mcHZRYDuObY~>=B7_h+N+&ddBuF21L(`DKFE%CU3iY|>pcZ&!RD~*yU%*UCYU21B9(IXzkR*PfC(qc#+-oDkV|+nY za=5fWH@%@}&R{`{okG+i5Nu3`jH{jedu0a3uOI$0wT~ie==)ukpp!Y`&lsek-|Uza zl4U<0c9t$BLext70IL+jbY*Pk;>6phuO6v}5e?45yVXcJOeG*liYo`v%}6QxApkF^ zt5TvXO~uie5v1{Q{ujnaIa(X9mN9-S9G}=wr+Iau>RmQuMRjL`GD%&nPPlXr7s$`L ze?E^=qmS;b_7v~?Hux&Jcn4*C%(bMbe7`#1WMboitBqIIEA*2etPW*E#n1qX*pW0# z1uxI^F^Oi$l1Mnzj-2}GFU?bmqXlC~zW$ILr6(&UeL|q)8~2{og{{{othjUr5JvT; z!7JI(Do0Wl;1E$U{F3+wtCu-}V}B1kDLU^PReHIP$NxW$NB&5TB?q;As%@-I%Vj!t;n)mzkCKBzK^B~O4YZr9O>m1m-;+hCow;g#Ju18yrH5*HkC zXaUf&Ut z3*eB7e#?qZoj}>77PA_}*2e?^fe-hp7Ba%DBTN%%BJ@tnno+-uPOP>=ziWUuYo%ir z6*d_8)S6A~3;5)b5YHTj4i1KJ*|>Qji#eG0s691yImLBF%;u1pw5Cy1pJ>{Yb#rSt zZ9({>{?-eL35Bti{&Ms+~4j}T8fEbrmsB12BzQfr;#+VA3_@yU+F{= z$`Xh<2yI$eiMz6(3aA7zhhXkjCmq!8Vh{ASE{4tW*qTgKWXjcA`kPeR%J*OdHBq$! z?;tSlICAbRoTOMelN?N!=haeJ`e|GLKw1Aq4~&|TIJU_g8Kj`vX`V)2snV)IO#kdNKZjDScrfaglAca5!iYzA3$v#xL_B}0Xa!`aknZQB^& z5DE$x&Wh3WpCc)=FhVcpB!Usw$Y%+Hv(w{73bG307K#_lImK)r&5`cYrLNuEwtw1- zKl0O3t1;d`3@cgT0meS5-8)4O%hl}nfu&uma8m|T>`SCu-{WEN*7ia#8TkR5ImL;B z07Wj2vE#Pma}E!TH+(l-rSpA8LhmAG%WiB)})vn(C^df#q`nKu`t-=Nh;t_bir^#mJb zyNTqUUJBXPkI3BXtO$>P2SY(&wt)?4zK*VR=E|k4zkZS8?4Pc@X|fx6IKOtz=4nnf zzmZiXrSl~i%-FLo+nd^64Z|H(=h~!H_dq3VXkaJNLqDR7Z)gkRyO_n6*9)2=P87kJ z%Lc=E``(I+x%sXc9Q2lw1;U5^cCKi-_H zubJXfGY_V(&PJ?`!)AkQRw|d6e{OblZ*L!S{{5@$htEMEouPX%G38b**eHDB2c7 zd*Y4$1G4&CY}Ve-?t6<`GyZ#CGUOz!kjN?jlPs50jFzt5&y5d5&MtF>4W2?cW@^YB z+?clCR~6mek4VEuhSmKg`M&AZoDCbX1OI2`&k8Jynwrfj#atGBQ%4TQQr-=Xe)m>_ zTXq`?R>~=SHiYgoFT6jvT6 zg|1>gvB^V9^gdpne_j9W_AhMV37t71&2KF+&|O>OBH7W+bRo4o?DCDx;-43n;LWtd zf^BmZj!tx%C{qJTfeYq%`+Gh;o47FRZua54Vs53Yz;|3=mX; zus%>R8_~nvlw=kFE^gu#LYi;$j0Ae>O6w2UI3@@L>9rG|EwG;`a@GEhr&S&d8IeL_m&o zHxmXlTIFo?O`iZ`veYit?-=8QR0zRbCmFrQMl%>?U18a>GLpHNA{9A^A1ti|vwe>d z`2N1Zd4bLIS8XwF@SCnHB2lR%__`_BBR_O_J3kNoZuNPY?UK1gLL0KEv(DCXJc;d~!GOiwSI9rMVwMrhw?S5B|9-#>0_%M*Svr^do&YIlNp1|;1o%TN; z+H1^7)yUxtc$Dh|K8Q~tZ%)P#0?oi?>7!nQL>%}1)3!2UXWqvojRllQ(|yePQl`Ns zr4-wB%Av=~4%~!^ffpo-^RyC1D+3A~J&2yZX=a|5;sO%7Ni$b@@_Gs}bBXpYRf9Bx zX>Eumil5AQ4wUtF;1Oj3CLrL1%hALTC5*>$fGc(Bkl$&#*V5Uv`xG2Jft)+4(q| zdS9*H6)jAo(>XH(lPx;6S*8r)%aOsG`ERpn-2%jKSsr$JZ?Dqc&&4LL-meSvY8%ef zXeMO&&)qKEQfn#m>`u8qy1EM25+J_~HpQoQ&F^!Sx-G_mwZV(vMPc=C*Ovl_H2o44QbCzm zrr7O!d6^>K31vs7MKcbStK&93;Wl$D5m|`(NyeM^i*@j@^UtP(pfs9z*nTG6}HBS^>H=ySW!Yg5Wlh~NI|1?TACP~ z@I8v&eZ$Q6oXqLMEI_%w-iN^qe)Co#Owj;xbkDK63{%DQ*$YvT+B3KTUC1}iy40yl z8i!xXq~G7bDk>V9cevWI2dutsw%m6A7-aGBOxU5pAhxJFGd^b&*6D+sEll5CVQ_Nt zF_lasLN&{C=X=u~2-3KPml>Ec&a1vmSy?Uq*GV2F)#UJdgevEE)yaC?q*Y(0AV|M+ z=w-u`PWuLi<1feCnMbEyKoyz1R5T5?bEblTF~OE~AvWqFhXTPNpV)?hEBwe-S-Z%K z*;^ET(rw_^gjF3ud}hPhbP7IYwJ5HmV*Gs9Zp2USF6Cvl#f7L2z7o^NsdW{HhBbU{ z*M9jh%KxNs?5$lv(r`7nt9sF-RvWAhZ*nzqEss)|6xn%MNKDrvH;N9V% z=;j8ax}xVYE#QsbAxeGCbbiiyfB*6!E)5WQ zJbJDrWiogbKtAgLNvK@#c2ZIc@#|cv*<-n=+^|eB>BX$m%vAU9u(CW6-O2_Gl#M42 zbF;b|19-_)oJ(3kB_z3a4WI%!a=qtct9-5@;MhV8X-XU?gc0w!$xw4w>B*3sX#Jtg z%2(xVX%JHNo7C5NUS{&na~6SnO!(4Dc4%(LhV8G4T$iqL)#Lzla_7Rpk^$QE6$$j^ zBGtMrv!DyZR&x@`jY^A@(jz6mPi$qFRv6`M^W7>CLn`qn{yu?LoUVo5F~YluOaL<( zG9Ke^P?rlT1h~iItBs4uCY3|DI-lxJZ+i6H%NS-Sy}*B1%wW9qs`dzS4&r7pQ+^|( zreD2xw#R&Id8a>Lk-sn5GdT* zKhx@{=D-kM;Hw>3M6TxBlN~H~j`9?IzaZ@_ua`$-+>@HL=gKYJoTUj5Pz@K-QX9CP z3MWO~`mn43=<8u(6R39?6xh>EI3=ZaLsVYH2(?hV`29bD`dCDikOU-Uq$34 z1QXtn_4u2b8cu~{ zdPPj#+M&5{T{*3ME1uF>lfEsV`njR6R*E(7obu3@1sdf^EC|tDC>4*ZB^9Xu@>TU; zQ8ig0`~M74>4am|!tUvyI(gxmiDcCdJQr@5F)x@1v&<2KWY{*C3;GPwiM(v5Kwz*d zr+TM28OSMbr0w3b!FEdWx6f}ItNbq@$1IHh7f*i{g!t!3H*6}3g{=HeAae-G2|SCq zm)W}2SLg}zqa$^B#z$L83sDr$oUxlv+T};}-tnLU-ZGqCRWd!iceeWnoHj{B&3wEc z#$$h6OFcmNnFCBmk#kHF^#gqgEy`cO!`leo>p57;m7$-7^>{3%iE91ylle0^7g*k* zJr5%e5G;U7qO`rksJJvgu@`ejV56hz^5ZZ}RadoP)%zu#%ncxBI`~)&8!Rg8nqO;} z;_?7tD8-jppGB0aoE^~B;HfriYqym&C`sU(U7cO4@vZt;ij=ZKCEEVTr*^s8VNo#( zZ+yeK%EI)d9H{ly!LK;<@@#+#!FNrE6y#+WO$*4us%O@2hPHr`efu95SJ-ql9p4M= zcwziw?WrA8?9TRwok?T}5(!P)8BKa;Y&U$>VH=q!Dmg!N5RqU{CIqtc%1t!@DyJD; z4l>Ryi;`9+kG;OBSngRq`cxFIJoQRim zK}LMAtyZupmyzwJQA^uszbgX-8`YS49yj!Pl>eSFO!eQ%B>ou6;N{jAKzV`Sfh=Rz z2h)B+t?T<#rEKfhp&BWC1)U`@>x*`!RiIKcrdyjw&Li#lsjxUNSG~NjfMOF1uWAr{ z@wOVQ84i?Jk;Mf(8;yGO0FdLo-LEUjLTv4-@C}L$Tdbr%&4dqzU-|8RT>!QPR(`!n z6rt!_Jqk+$QR|)u?j*w`hDPJ zh+6QnO{JX}-UmLAnv}F7FDK%(eZQ^62p1oqL74Rl=A{x4`)}!)79V$g4`0WY4XnIU z)s5dc0e-fjRfJLg<<7gFmvgD4${Y-CGf~&)Rkp9z&wK4cm=f!P!0TwC0#w_a1lVp? zQQ2XIoXQiu14N8MV~BvGMlciPC3!HVWI^W&)Vg&FdzUpl7vjDUcKCzJL&wVLUb~Hm z;XD5-JLmJL|E=o}RLH+R4gA$)AReQ~>E@k;4Wo&B$=MKsO^0~<~^u2q*X1S)PGRpJ)RpD|ON9*OB4kr&6& zIQU~;qJKt~Cy?ymtvYV3pEQ0WyUph%Z*h`?f4*=Ri(EIIvAmp_(2FdpFWAF>ytF+y^IBi$gEZ=4?elhF@O`6} z&bA|G7bn5Z<=FZ*C;qqG?B12mv5#H%-J}woxl#a&-kFI6o-~xj;OW3YDH~d|F!($9 zUP6GXW`Qhj%)U}Hor3IyoUO6b?8aqmxiVv~mlm}ww_6*t6L2EQfPA!zSFj3)eaAAL zgE~;nGhs=WVcztd`?x3!y}eX><<`+`)}44w#hMba?4h|4Yw<>dr3%XkQL zftXnm!Gd*;r5V9jv>LD7Jn7>D<8ga$HDv$Rye6)$~v>As*ophJNGOJ9j8M%}aA4qK2H zNb@O=_k5;fKi;EN-aa{+s;!g5l4y^iML%8mzV$)n8yWDOzOf%i#<{nsU69B!{Rom= zMhaEc!`w5&pak8~J%qm2%fh=*DVr-b*zuge zdnEkr3`A1_Y`L|J0z)_PY2_(EOFUNor8hDM@$)ZiibOWH`(~l9vp*2YI~6T7XvOdJ zcK-8!qP#jj+nxHXfY%fs*xt_}Wt{o65;5 zmPeXad@(}!Lu!*RhVjIU_VOfQ!rq>n>t0@&85pFQjLFdx%BPphDEp7T^# zfX}YZwKTn>rLIh(O{Cl89NsckU568U5TtOKdxiw-ml{NG;y(}Te z-ZFs&eqxRym40*aLtEc|MjU9O>ev-SXHjVaBSLy=}>B?MdByff_?c6?(t8rie@!%H&V|j58%x!`?^w}j6Q!Pbw z#B@!XO<*Bg5-D)w1Vw6`tU}jrG519LRCOj0R7m4uq4RyPJ$>}H3R4vNM{27^(Ya@V zbS{T!Cz#*pacdezQ%0eW%+W$r6N#0k*auJ-mwd?Fc@>}7mhy2qp{cff&ox%O?N6i~ z31x0WeaG!6*_n`It)7ozFOM(GZA~5oI9%GEx}PMnK4Tj^L%wc$`P=LkDFp;K>(`>=iHctdUxuqA`el6gKtNBh0qpLbCjojsBIY` z4eRx10{gmLscR7C=x}ATVa&VX{+^1`Tt7v-i$6&WwIy?JRJ)SthrKRO_? z76lRw(`PLC26~IE9j=D7uI%yJDJtDqA;PvvM00(G0AfvIerETh44V?2YNY_*Ldumd zw*vna%m4q=y~cl!!q_HltLeFKPklE9F2*Gs9A7Kl+n0Jh^HB$C`I8UXK`Mn*N$+!h z_XebK(pxcUC4n|jh}D77HUPQ8P@Ew4$n}>$5AR?9{!20>SC-@6_)yE1urd!~`tCT~ zD0qc)hPcAy%B;45dt|SqB_P@0Lvy)jlDdYCTqjT58{MBf*agB*!o7oPLP0zUWnH=6 z9!(|QdMN#i**tODk;Fx{JDxmQMaIAJ#|{@tdBW@6E-!lV6P{w{6Gqw=pzeVQ1Pd31 zUQ@Xp>5nKs^Xf}X-`2P(Pf z#?{*j-ClQJlvn4*Br5U?dvz0l_9=mC1?ln*utj!lqOJ76L9D~=$5S6rsLggk6FS)z zC(dnR9Ik8 zDcI7jy&#?$=OodKD@><`ji_z}#%_G&-zjbl#kP7bUll6m_Jk5pp=+J7f0x>FiN4`y zXK&O7s3zjh(37FjxP_wJ*qlVoL<^^uxt79~HA)K1!5~wDL>PaJC69b?EPQGda;d6a zEx>9tBC4;RQaKpoCpFqchiB{K&RpP?w8G~BWxOkrL`@9bXF@mVHsd79jcQ#ex1byK zs99Hdm{fbT*xycFfFJZ1qUl5M^D+@BN_Aim*cf6cNz}b>5Y>EVkJMoX(Bxv6aio{p z*VG?z-N3XZvqUQEbhXv0ynMzBQ#hekq{UMxJ_Y^5EUYlCA#F%Z#s}or zYafQEcSx(e06QfwvEz=Of@WkA_P+u37z)48h;wpThI}Bqz}+OF*Pcg+q4o%`DsdQ7=;afPI1zs;-fOITF=e6mob!09PlkVB|BK z^|N3c;&HB>qPuGFb5GRp9{hC=Hy02C8qAm6Qt8uNgU1n4on~VS3x|e^#N^2C-l#c~ z^4slIGCkMsliiY%s*()@D`u~TGrS>Z^If?_e281t42}Yq#>@KbMs^9S>2Xf0)1zh_ zkk_hj2m}05WKhR#{*%Yk)m({WbP>Z=S zdmO3sgVp$%gVKWdMn(CPx&8BjUhI>Hj{dDo4Lfcm(l*axLsQgJC9%7_h02Oo1dFH( z?zM?2tFnVRJIL{683(5Ki!_%8*>WL%I(dM(+yOi(MObW`(kpZrvRlY9<@2PL3vmO% zvw`7DV5PhTy05L$3EeE`7t`BakS1I3OAPi!g;gJ*z-0C$!6>A5}EOCY}PvPpGSFN|}| zy`24g9ex4!e8}S2PUQ}`&+S#^mnA)c^Y`FFue@PPUA>Zog}m{E;PAvE_lX{(wGgd5 zMCWP2A~Frq3mL`bB|cn(N&A3Xu=!~cL#@8gG749i%Lz>1G{DOJMS-u(&<)=_R%xAX zcXnz!l$XA*T3xIShDd(9n!xGeQ1AHQ_*KukK{3{wCr$BnS0bo9FS&=GdKscKN}h#% zPUKrhugj&bLhM{+H&_~_omh-#`b6fbg#3e~uAh7jeM>4u2>a3FtSQMxAT3`yP(%4v%QxRcY1N5^YS&Vimf)u*VyTvqKv285<>l@C$-jVz>&sTmI_0)m=?tQtw za4nEBf$y_@eWb|0u{goThoP~t{ZP|KQcyR(zxSXRDw4VrELGwXuspO1VyZ0E9A5^S zS~Z!5JFcbVvZE)4X8n904Zg5VB%Co=v_)(7X^J(1O(k5TEH#jEUYK3MRb(1=o|1!; zMl2!$-4R+h+fioes_c>dUGO)YVzfHx{^dtTX~I3_%W@uRKgqio+K-j7Vq)OZDJAM$ z9CF$zz9#zVlx0e4NB1wI+lLnisy8G9h%pC_K~2P3Q35Vwz1eO_Bn~AqKW(9%ARGsH zDNq77@f3&~4Z2<5Qd)T66$5Qi`gr?hUNOGBOTwivzo5#(Tq(a08G8wTVrW4yK*g-L z8XRX*AeiWUS!E@D)?cf8w;rIHp_kht+V&va7r!VFZ8-;1txi3VZC^<$N!<(8$h1^R zZlCWvt>kWKY|JmP5)<4joRShzw*fowOozf$c3;kSZGorz&!v1-)#q23VA*<7aia%y zV=_BD)YiHeZ?64TYup}AuJ74Rob(lpfSXSkK)%(lgGWbv@We|FXoCd^$ao7=b`JW-`C6}q~!UT z6D?g~PRF@`Wkv4kKHsdz7-fVexUI ziqsNR;X+a>Dipw$mqnm+8~N16SWZourNr z^z)j%`xCU4>qC$4|B?cR-TLGE-{=-GwbsB5jwotz`Tf4h+nr*!-jnN^nV9(@?A3|q zwhx1tg>0%qzf7g#AKdNrUTsENx}=^*&F%rxGvC}UUxKDn@lW`h@1hX8G{r5+WfTG; zI-lWdyr#qs?}9__NLFen^s7fbW-q|v3#n_t4jR^8fsh=_e)Bu6S~?5v)*;hDLN8ks zS$={S(3z*_TV-QWn+x~IB7FP&0d@qtL=9_2bM3Zd3#U>EzG9?80Esizwq_NFbJM>p z0}PaxZJ66#^uor(kyAJ33FUUMkVmb4{m;-O|9OOlZ%KAC5^4>wH+?Pz^eN-p#nLb2 zdB?mfC0HbIfZTn}NL~WD^kUoVRUW1nQr$Rg?|f8!m?~Vg1FR3K=be_rTYH50=-cWv z!Ac7QgUvXsYsx4Yi>KO~JG+D7a|^r9&ALjyzc?twD!=Mw_FM?6Q*S4v#JUw(5x~{< zC%J61(6#_yGwM>>lF&{&=#pJ$miyWT?Lk3OEd>-<&*NPsP4vh#F5vDQd!LkQZ!ijw z@Ei1^Yp|;_0T_mhk5M1-(gGK1Zij+TpzJKy5l?X<2@#Pu+gZ_6HSdSXttq7~xFH*J z_oYsiI$vGk9~Wu^NOGNflySXEGlZEqn!y9SLyZo`3{>i>EC6I}iSY2}YO2|9WJ^xF zJ$LVSl(ZQ?IXPp~8Itc)W|NUvPa+lLXXdA4oDL9iMx`x{vc65|Dklcd!uCQWz}NEtm{+~RZ}J_M;hLpjwAz>@vbzB`eFQuwm4NOumZBOS_FjD876axg+|!sp zj!kj7-S=B4buMVjZ@gjBU$*X>UR9I!N!w`eR~R7Ul`2$- zMYXR<>LDFErNy2)ztpG#SQFXRa!NuHIt%(dA!Pr2OM&_0K(>`dv;tL)KcchPdPb7iq}0W||qRU3AA^ zspDHtFtdxEG44gCv4&k*6-l_~+iT=5Z)_%D&UA80tLR*N`u64WHQ*DQbihdFo>OVRmTT5k79xM5`|l)E(O7ME#74CKDG|H#O{&bxlgXIF;Icbi zrr-w^j4Q)lk4LsyQNs(epHSnT6tyv+&VhsBz0ot;0Xd3@FA8s8LjaECcW+YF%;wJD zzmMR)w*%?t!g_(>7=Q;@r zYS$m^zGE03ar#ma(7~8zbEP%k%YuR677&f?toGeMz9PTRy)7=S)7<-n0bVpAqtly| z6bNc51?%KyHbVi+NQ!U6SGin9gHAP`b1$=W5Lm&g)UlQmk6)KxD6;fRnmM-nz1P#T z#Oj9d+dT}g7PH~_n}Lu2W~w;T&@#wER@r}Dns*9w*2Og?X!k2_9AKBynuVF(%Ghaf zf3&JN9Y-OBT(F~sx}qTX!A6$e8<*Xm-dg8pxg>%6xqrK5|MhkLy%+ybmj4$(Zd^ZO zQ-5`rN_iqe{PqU)n%V#lLnM^^NUxUz&j}(6N*^A!&>TEM!NVknS|fMMP(&of=(^J) z7ME)o$oLx)?|*zD{<)`r=(_#qvN+*DyAl|Z&U0jpEgybQohA2hVM24qhV`H$&3bt$ z3Go4C1Pj)e9>nx5*)w^xZukaK%#!kKQUbpmA_XuL%owGH$XOViwgsxG=ZOWL>;k+B zk*wX4%ORn(6L>q@GHX{Bz$M$>Uq<%%-y+5TTn+#4slJ0;9PKU<`2I&zo)T9Jw7=BI zV(-L>Z#HlP|9p_3izU3Pv4pYjZ+ry5Mw@>T?(2KNtN(iq{)H6(hB0INyD_0|eU6#* zZ@m7{m->I`=D#ZCbeDfM)8`U_K(P}qEJ|I*7NX0_8sIG7?umtO`WBj6&+mqyvk8Oe zu1avSWE_!4#)`V{W8xvDlbVhevx&wl_Bz%N#w#kr1o!R(5912*LL&HgS#m9=IPO6$ z5(b1L6|F@syShq$VoR!&YWVJOVJboYxS%l^M-!|2#HK@PJXoCl><{Qq(+oP0=7NJ4 z_EzP__X+bFDsXma=(9dU$uD%Z@4QJgCRTsbAW|?)QNbz3?D?dF})CZ z)$WCZtgY*ADSP7zNHk7}a}&)Mvf5GbVN+BqLci?Wn* z91d!QOgyp%d*!YYd6DGz5PVV5MC2if-DF^?exd&Ehh5%Sr%I;gSH5JkA)$vvGr4i< z>vrD$RAF7wBf*9U*AN^Y4W{Do)fr!#EqqN5AI)46#kx?B$xX}y#=YZ3xjw2rb2ptZ zs+vw%8eb&e?`T}9$tNAZ7$SuSYM|hKV;xUyBDWUJHY10{UtNe}xK}X56XUwP@?I7A zK>=30oSa7kIoEgpGFYc_Uw9lR&K6F0Ry16!-#V00UrwUi9*rd+p~qMa(VU$R8KSqN z^)egZbsPGVn#Mn|$$ZoGuzfd!_r(g=-EVUHm^Niy8Be@+fZa;)d;#}+*eY4j0aN&)Frki1ryxon`g{Ax@+9yH!MeW`{&uyL`VdfpPQdAYxB zVdgd1Jiql?vaa;1WuHWlYC z&)6n;nW|}3Uo#ffL*E94L`({mo0TZjt!#O8@CbP-0X0p=OgUM9It}%xZio&M9y}$7)sgzw6ZajFmc5au;)QRa3 z91_KS)W%(O?-SeWhy&f}iaJXZp&j#)m|@UpQ)AdsQ!#{fP}S_{`F@eqC83@`CRDXJ=T_Sy2zN{az2CRkJp4l!Q-G04CF_#SFDQo*7&qml8f3j@WUt_Tz{!c#aqE@i zl0_2zGD9r)44xGi<++sR_~W|(mPkwup)D%rh1WpL1fBo!3uw29UkP}|m5=;D4WJu# z?javhBck61cQ4$~=Iot9@~Gxx;l06LDL8m7?Qk?hvUdE}v5j=0vu_s$UYu`|th~Xc zVkzNT53uxE_u(~TdslY{mXfFA*@9jaxI4Ea=TU4*GLH-;N_!deGS+KEUaSbE6dqeL zntX%GWiHo-8m-8M-t>;0>ze2~1cNz<$r(()8WHtFxc6$wKn2^rc?P~+liyG-)Kin$bu>AP@ zR^`9hieONcu*Vtq;Z~#iy;q;rsIU3oT>JH$dWH}-E- z=WB3rMlVe4?MZhec@pmV@VO@9!6IFNbJ^R5^1JPAf~cEy!Y(MqX2VGLc#R!d^8#EQ(K%3F!pm-K{Fw z$yR~MDZ|Th+2nlOJ~WlJ+H^<23Nmad4y}LMSua5`5MlNugjZlH+~O8_}^?T0!-?Tn5a#hIJXh?ISh+%c+*t<_doVr@nG9J z>?;3_sE@B$0%hkV<}?G@jdA&H$YNwr9Z{B_?8%3+_;}J*b<6Z7sD8Z}h8&lu{YY`{ z8@7JBdlvKBSVXn87mypG`2Tx+IO>JcdZpE-_zK22nCs#_7FVw)vH;cjz#+gRqF1?8{G(U!UjGE|ZWl+U&K ze&~o@q~J;!&4TGe<50SJ@#ek2p|;^B21juC>eykX+WP56tWfLu^2PJPrGwNZ%Dk`- z75jwbb#7;eOHM0Dql?kbtAeobWlFq^A%3+O_t^hoBP&jagotR0`;T<{>nPV`I)tZx zKsK$EFh9Ed0$*Z7VB*8cl5^1`ZXQu;ziDwE1gCgt>=H4&Po>7rx}XiEfV{aGS3c~g z460t`sV$iu4f2l0Q3M98y++~XckHiS)?TDq;kdF4N}bgF-c()lxISXE@gy%y?0|Q> z-?rhHj&Q#lJg(0YP{hMD#FTA(j>`nbFF5{YReTwz9@p&K<}-2>1Ivr5JP%`x@;%cf z!}e0#f4KI5jqOw#KBW5Le8hM6X+C;&atv3ppm1WqmZmPHsH}yy6v*O5m~P%u{sA5d zRDWmx!=S{e$Yf`P*N#Y*snlxqU=R4L+19OrYHC^8p`{P({d?`ModmR_&jCbeB)ZW%%onh?>LLSu$qi(a1=^ zW3GNYT7$~_!fq~K8?G5@GVJ1Vu4op$p;9;~5d^K~d$(r&@~foTnGrpvW4Y&18re2SSn^Ba19umw*dqoX_*W+zzLXd5UqU<`ZgvYJDq^WI@j- zMAe*i*|tb5jtH^K)4}E8GODI#wHYA^X4Jae1;W-PxIor7wj~0^HUPPnpZZEl%N&N? z^(&8fOzRI1nRvuVh8L=D;_YRO9|%`%oEeh#^R!8upo9HD2v5}r!(QNsET5Oap~)@w zHUwqg8gxeCCNJqie{!SZ>09MW81a3)0L8C2U#pxo|G2V5D1O`I{TZ(N%y#EZ?U~7% zYzT@tgD50*alvqmmDTtR(xu`og5K03xfGFOIt8P;a@+ zszU}eq-KDG(vdA3ASr2UGdRu@@6%_Wc%0^hBsFc+954k8cJY%W`%eoU>%r9H^Akcp{dnfVyi;4bSBr+Xt!qqqKqT;;P8`Nv z5Kxv;VNx=(nzu(+otvHA-8N()%Cn9XqVBhvx;1~8YE5nFgkL_hm=?~vanSK=iOngC z`YXxf!qG+4*TNB`=rqOBX|Qp^t(~)MlZFx%_m}LOlsR9b8%zs-cqrjS?x*RS-AP(y zlnM2ALGy?jT{>l&smjb!Uo>sMn9q=f7PJ?;Fmh9A>E5f5T=X^=zP0H`BdW5e@uIcx zgC7i()9mdd9^YC|gY!wuDVkWsmn3kK)WluqukLa2tF9cs4Lj|2ldI)EpZgPg^>|ZW zlLH>6IvA}e+cHoeLdI^@0JTlrh10u6Lw2EG_1UF$VcR7KI(&pxx!5&zFX*#xKANzV z{LWM!tP-r1JIert_p%O&^nu6-?+6@JeU~DdOgUNj_-tsA^l>!D+WcIxWm^WSa)d%s zJq+F1jp<3#jh{f2_%;zL0XC|kGktA)qX+B>o14^Za} zaoalNXz4IR3*p_lMxMk6&b1&j{dST~OFALaJV(V@mlTJXk8OZh3AC<1ZD}d>_sJL? zzuSvEtCT(ra6uX#fx}`Y`pJC8PV(qJJQz0(WKebw`h({v?w1z< zPt9nSZHa~_a5fax8-lTeUBsLm961SdycX``pu;Wv@2+rbwmo$X(XFT}UVq!zEalE^Z7Np=M_;q4#H>w9c|;w3 zLD$VRuvBrV~TH2VneteI9XeKSGf-apA=k2e(9Vw5At=Yz<`K(=$QK0a@@e8{) zrc}WnZ{v`;S30gJG?JO2^Vv^gfx&SV@kNsz^{tK2YV> z<-2HMp~0D(jiM6JL4l)0LGNhZ`TxW!|BwHbwFmfn+vcVXjklVK{m9SXdbi|ZFm$i z!KoNWCGE%*8yXC1J_-#hwI55OpJS*>(Wxjf5DHKp^r_iF0!OPe zWst~AGDeOIB_x3*i*%Yk^j?0zlnWHoNfQhJqO2gx+HhMuJLV)x!kLkE!r(zy23?A4lae$lE3QyzbWB4-_44;8nvcx4WN(hshHLDBottOf zKq=!YGZch9{h5t%ipP|$H_D-Ov5cc+<@%V6f?rgf*`%q?MyHJ7ya$wkVB!IDu3+1{ zP@Mz`uvzyac+_N^O?BV7H+}N)!)OJ8oMg0I+xGs@R<^;fR-v+<+|mv&NFAN5sKV=N zzYTy24o(yf%QUgTE#U(0R;@|^gBH^Y&x_LGKdBC+9!>qW+2PePEO(pvr)@ z9Dk~&-)8Lq7=}mN%Gss%{XDyx09HD=1g(o6XzPi@lc~R^+0^z3h$u){FOkY0J&V#? zz>6j00B1s?biud#q}K|j(cG%hL0|@dgSq##J=xLnmVnNT-HrYHT03X7L!o3{MUoOr z3_?l?nqqFQM5yM8;ZX&+DQ1MSg#2E>p@9LryAD&q>3kYcQ+6A!fZs?uNM!N~D8XVR z*Hr}M?)3JyNv|>m*ze1dDC5(mgLg?yM}kJc9+u@(r>=avAr6-h*_W{YwtpvbXf$EUR32+t2C1W6%s3M^MCzy zdI~eBXfq828lx{)6h{0aOq_@JT)<_{Wn0reIGPF-B)K_|w^GqCYP#c^ieBhW>xM616&b zrM1`ADZ*O+Y!9qqUizzdwbjLC=cife#_WmRr5fV8Gg>!#-F<#%Ps!_dR-aXQWG!5M z({IMweD(d|*=4VR=O9V2uX6Y}@A0z}ADwfnHqJAiiw>Z;xKzfC)80)B4U6LjXE z>wzyDcdyUY4VV4T@HmEj%YsjQho5aKG_IexX4c_`%f-Ob3I@_YH6mk}Rq#aWn{j+Gh*WQ1*dexz(c!1k2Twx+g&(&2B5t!_LA zPSZ$xefXZg=G%XUu&+_Cy5CH`*c!UO{)E{4{DT`rLxJs%@Dyj4lUrZAe0bG&@Z{EX zq3zdJecIF)y+?Mc@;)|)zDT+KnQKE0_AOn@x*{a{-F(MM)vsSn+qLTZ*}w13O^eGv zlwWa9u=QcdYtQy1v1k#lIjcoC-51q1`(f6*q9G%0g_ZXuWs}EVU-=D+f6X)rQ@^#+ ze(z0j(`CJHe#8o${5J9Qm3r-*N9NWyr+448T~;>Jc8;0r=Vu?D{av-H6&}*#VdB(UTSCQP51gWQ|t9RW2Kx)At4^Ceu@g0WaXba R5<9EcZ*dMq`G)`hO#qbj5^Dee literal 0 HcmV?d00001 diff --git a/doc/mutable_data.md b/doc/mutable_data.md index e667ec3d7..fa7ba6f09 100644 --- a/doc/mutable_data.md +++ b/doc/mutable_data.md @@ -9,6 +9,8 @@ For inserts, record will be inserted with `sign` set to `1` For updates, `before` value will be inserted with `sign` set to `-1` and `after` value will be inserted with `sign` set to `1` +![](img/replacingmergetree_update_delete.jpg) \ + When `optimize table final` of `select .. final` is performed and when the merges are performed by ClickHouse in the background, the initial insert record will be merged along the `before` record. @@ -18,8 +20,6 @@ Non Primary key updates create a record with operation as 'u' SinkRecord{kafkaOffset=62984, timestampType=CreateTime} ConnectRecord{topic='SERVER5432.sbtest.sbtest1', kafkaPartition=0, key=Struct{id=2317,k=3739}, keySchema=Schema{SERVER5432.sbtest.sbtest1.Key:STRUCT}, value=Struct{before=Struct{id=2317,k=3739,c=20488251985-66135155553-00362235007-72249840112-70784105787-84584360668-65106023418-49140058226-99031281108-48426083028,pad=18846546959-44726413785-66695616247-63594911107-83062207348},after=Struct{id=2317,k=3739,c=20488251985-66135155553-00362235007-72249840112-70784105787-84584360668-65106023418-49140058226-99031281108-48426083029,pad=18846546959-44726413785-66695616247-63594911107-83062207348},source=Struct{version=1.9.2.Final,connector=mysql,name=SERVER5432,ts_ms=1657658606000,snapshot=false,db=sbtest,table=sbtest1,server_id=842,file=mysql-bin.000003,pos=16210729,row=0,thread=22},op=u,ts_ms=1657658606611,transaction=Struct{id=file=mysql-bin.000003,pos=16210580,total_order=1,data_collection_order=1}}, valueSchema=Schema{SERVER5432.sbtest.sbtest1.Envelope:STRUCT}, timestamp=1657658607050, headers=ConnectHeaders(headers=)} ``` - - ### Updates on Primary Key: Debezium Debezium handles updates on Primary key in the same way as Primary Key changes. diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java b/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java index 374277b07..d91261e39 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/db/DbWriter.java @@ -86,10 +86,12 @@ public DbWriter( MutablePair response = metadata.getTableEngine(this.conn, database, tableName); this.engine = response.getLeft(); + long taskId = this.config.getLong(ClickHouseSinkConnectorConfigVariables.TASK_ID); + //ToDO: Is this a reliable way of checking if the table exists already. if (this.engine == null) { if (this.config.getBoolean(ClickHouseSinkConnectorConfigVariables.AUTO_CREATE_TABLES)) { - log.info("**** AUTO CREATE TABLE " + tableName); + log.info(String.format("**** Task(%s), AUTO CREATE TABLE (%s) *** ",taskId, tableName)); ClickHouseAutoCreateTable act = new ClickHouseAutoCreateTable(); try { act.createNewTable(record.getPrimaryKey(), tableName, record.getAfterStruct().schema().fields().toArray(new Field[0]), this.conn); @@ -99,6 +101,8 @@ public DbWriter( } catch (Exception e) { log.error("**** Error creating table ***" + tableName, e); } + } else { + log.error("********* AUTO CREATE DISABLED, Table does not exist, please enable it by setting auto.create.tables=true"); } } @@ -113,6 +117,16 @@ public DbWriter( this.replacingMergeTreeDeleteColumn = this.config.getString(ClickHouseSinkConnectorConfigVariables.REPLACING_MERGE_TREE_DELETE_COLUMN); } + public boolean wasTableMetaDataRetrieved() { + boolean result = true; + + if(this.engine == null || this.columnNameToDataTypeMap == null || this.columnNameToDataTypeMap.isEmpty()) { + result = false; + } + + return result; + } + /** * Function to check if the column is of DateTime64 * from the column type(string name) @@ -235,8 +249,10 @@ public Map groupQueryWithRecords(ConcurrentLinkedQueue groupQueryWithRecords(ConcurrentLinkedQueue modifiedFields, + public boolean updateQueryToRecordsMap(ClickHouseStruct record, List modifiedFields, Map>, List> queryToRecordsMap) { MutablePair> response= new QueryFormatter().getInsertQueryUsingInputFunction (this.tableName, modifiedFields, this.columnNameToDataTypeMap, @@ -272,9 +290,9 @@ public void updateQueryToRecordsMap(ClickHouseStruct record, List modifie this.signColumn, this.versionColumn, this.replacingMergeTreeDeleteColumn, this.engine); String insertQueryTemplate = response.getKey(); - if(response.getValue() == null) { - log.error("********* COLUMN TO INDEX MAP EMPTY"); - return; + if(response.getKey() == null || response.getValue() == null) { + log.error("********* QUERY or COLUMN TO INDEX MAP EMPTY"); + return false; // this.columnNametoIndexMap = response.right; } @@ -293,6 +311,8 @@ public void updateQueryToRecordsMap(ClickHouseStruct record, List modifie queryToRecordsMap.put(mp, recordsList); } + + return true; } /** diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/db/QueryFormatter.java b/src/main/java/com/altinity/clickhouse/sink/connector/db/QueryFormatter.java index b8b4aa270..2b786548e 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/db/QueryFormatter.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/db/QueryFormatter.java @@ -122,8 +122,13 @@ public MutablePair> getInsertQueryUsingInputFunctio } //Remove terminating comma - colNamesDelimited.deleteCharAt(colNamesDelimited.lastIndexOf(",")); - colNamesToDataTypes.deleteCharAt(colNamesToDataTypes.lastIndexOf(",")); + int colNamesIndex = colNamesDelimited.lastIndexOf(","); + if(colNamesIndex != -1 ) + colNamesDelimited.deleteCharAt(colNamesIndex); + + int colNamesToDataTypesIndex = colNamesToDataTypes.lastIndexOf(","); + if(colNamesToDataTypesIndex != -1) + colNamesToDataTypes.deleteCharAt(colNamesToDataTypesIndex); String insertQuery = String.format("insert into %s(%s) select %s from input('%s')", tableName, colNamesDelimited, colNamesDelimited, colNamesToDataTypes); MutablePair> response = new MutablePair>(); diff --git a/src/main/java/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnable.java b/src/main/java/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnable.java index edb181c05..ad0a83e3b 100644 --- a/src/main/java/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnable.java +++ b/src/main/java/com/altinity/clickhouse/sink/connector/executor/ClickHouseBatchRunnable.java @@ -92,17 +92,21 @@ private DBCredentials parseDBConfiguration() { public void run() { Long taskId = config.getLong(ClickHouseSinkConnectorConfigVariables.TASK_ID); - int numRecords = records.size(); - if (numRecords <= 0) { - log.debug(String.format("No records to process ThreadId(%s), TaskId(%s)", Thread.currentThread().getId(), taskId)); - return; - } + try { + int numRecords = records.size(); + if (numRecords <= 0) { + log.debug(String.format("No records to process ThreadId(%s), TaskId(%s)", Thread.currentThread().getId(), taskId)); + return; + } - // Topic Name -> List of records - for (Map.Entry> entry : this.records.entrySet()) { - if (entry.getValue().size() > 0) { - processRecordsByTopic(entry.getKey(), entry.getValue()); + // Topic Name -> List of records + for (Map.Entry> entry : this.records.entrySet()) { + if (entry.getValue().size() > 0) { + processRecordsByTopic(entry.getKey(), entry.getValue()); + } } + } catch(Exception e) { + log.error(String.format("ClickHouseBatchRunnable exception - Task(%s)", taskId), e); } } @@ -163,6 +167,10 @@ private void processRecordsByTopic(String topicName, ConcurrentLinkedQueue Records(List of ClickHouseStruct) From 08f9806916032e4349dbd8c4b097037ca6eaabe0 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Tue, 2 Aug 2022 17:46:37 -0400 Subject: [PATCH 12/13] Updated README instructions. --- ...ker-compose-apicurio-schema-registry.yaml} | 35 ++++++++++++------- deploy/docker/docker-compose.yaml | 35 +++++++------------ doc/setup.md | 34 ++++++++++++++++-- 3 files changed, 67 insertions(+), 37 deletions(-) rename deploy/docker/{docker-compose-confluent-schema-registry.yaml => docker-compose-apicurio-schema-registry.yaml} (85%) mode change 100755 => 100644 mode change 100644 => 100755 deploy/docker/docker-compose.yaml diff --git a/deploy/docker/docker-compose-confluent-schema-registry.yaml b/deploy/docker/docker-compose-apicurio-schema-registry.yaml old mode 100755 new mode 100644 similarity index 85% rename from deploy/docker/docker-compose-confluent-schema-registry.yaml rename to deploy/docker/docker-compose-apicurio-schema-registry.yaml index 3211ad5b6..a6848b220 --- a/deploy/docker/docker-compose-confluent-schema-registry.yaml +++ b/deploy/docker/docker-compose-apicurio-schema-registry.yaml @@ -79,18 +79,18 @@ services: schemaregistry: container_name: schemaregistry - #image: apicurio/apicurio-registry-mem:latest-release - image: confluentinc/cp-schema-registry:latest + image: apicurio/apicurio-registry-mem:latest-release + #image: confluentinc/cp-schema-registry:latest restart: "no" ports: - - "8081:8081" - #environment: - # - KAFKA_BOOTSTRAP_SERVERS=kafka:9092 + - "8080:8080" environment: - - SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS=kafka:9092 - - SCHEMA_REGISTRY_HOST_NAME=schemaregistry - - SCHEMA_REGISTRY_LISTENERS=http://0.0.0.0:8081 - - SCHEMA_REGISTRY_DEBUG=true + - KAFKA_BOOTSTRAP_SERVERS=kafka:9092 + # environment: + # - SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS=kafka:9092 + # - SCHEMA_REGISTRY_HOST_NAME=localhost + # - SCHEMA_REGISTRY_LISTENERS=http://localhost:8081 + # - SCHEMA_REGISTRY_DEBUG=true depends_on: - kafka @@ -101,7 +101,7 @@ services: build: context: ../../docker/debezium_jmx args: - DEBEZIUM_VERSION: 1.9.5.Final + DEBEZIUM_VERSION: 2.0 JMX_AGENT_VERSION: 0.15.0 restart: "no" ports: @@ -120,8 +120,17 @@ services: - OFFSET_STORAGE_TOPIC=offset-storage-topic-debezium - STATUS_STORAGE_TOPIC=status-storage-topic-debezium - LOG_LEVEL=INFO - - KEY_CONVERTER=io.confluent.connect.avro.AvroConverter - - VALUE_CONVERTER=io.confluent.connect.avro.AvroConverter + - ENABLE_APICURIO_CONVERTERS=true + - KEY_CONVERTER=io.apicurio.registry.utils.converter.AvroConverter + - VALUE_CONVERTER=io.apicurio.registry.utils.converter.AvroConverter + - CONNECT_KEY_CONVERTER=io.apicurio.registry.utils.converter.AvroConverter + - CONNECT_KEY_CONVERTER_APICURIO.REGISTRY_URL=http://schemaregistry:8080/apis/registry/v2 + - CONNECT_KEY_CONVERTER_APICURIO_REGISTRY_AUTO-REGISTER=true + - CONNECT_KEY_CONVERTER_APICURIO_REGISTRY_FIND-LATEST=true + - CONNECT_VALUE_CONVERTER=io.apicurio.registry.utils.converter.AvroConverter + - CONNECT_VALUE_CONVERTER_APICURIO_REGISTRY_URL=http://schemaregistry:8080/apis/registry/v2 + - CONNECT_VALUE_CONVERTER_APICURIO_REGISTRY_AUTO-REGISTER=true + - CONNECT_VALUE_CONVERTER_APICURIO_REGISTRY_FIND-LATEST=true - KAFKA_OPTS=-javaagent:/kafka/etc/jmx_prometheus_javaagent.jar=8080:/kafka/etc/config.yml - JMXHOST=localhost - JMXPORT=1976 @@ -274,7 +283,7 @@ services: build: context: ../../docker/grafana args: - GRAFANA_VERSION: latest + GRAFANA_VERSION: 7.5.5 #container_name: grafana #image: grafana/grafana restart: "no" diff --git a/deploy/docker/docker-compose.yaml b/deploy/docker/docker-compose.yaml old mode 100644 new mode 100755 index a6848b220..3211ad5b6 --- a/deploy/docker/docker-compose.yaml +++ b/deploy/docker/docker-compose.yaml @@ -79,18 +79,18 @@ services: schemaregistry: container_name: schemaregistry - image: apicurio/apicurio-registry-mem:latest-release - #image: confluentinc/cp-schema-registry:latest + #image: apicurio/apicurio-registry-mem:latest-release + image: confluentinc/cp-schema-registry:latest restart: "no" ports: - - "8080:8080" + - "8081:8081" + #environment: + # - KAFKA_BOOTSTRAP_SERVERS=kafka:9092 environment: - - KAFKA_BOOTSTRAP_SERVERS=kafka:9092 - # environment: - # - SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS=kafka:9092 - # - SCHEMA_REGISTRY_HOST_NAME=localhost - # - SCHEMA_REGISTRY_LISTENERS=http://localhost:8081 - # - SCHEMA_REGISTRY_DEBUG=true + - SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS=kafka:9092 + - SCHEMA_REGISTRY_HOST_NAME=schemaregistry + - SCHEMA_REGISTRY_LISTENERS=http://0.0.0.0:8081 + - SCHEMA_REGISTRY_DEBUG=true depends_on: - kafka @@ -101,7 +101,7 @@ services: build: context: ../../docker/debezium_jmx args: - DEBEZIUM_VERSION: 2.0 + DEBEZIUM_VERSION: 1.9.5.Final JMX_AGENT_VERSION: 0.15.0 restart: "no" ports: @@ -120,17 +120,8 @@ services: - OFFSET_STORAGE_TOPIC=offset-storage-topic-debezium - STATUS_STORAGE_TOPIC=status-storage-topic-debezium - LOG_LEVEL=INFO - - ENABLE_APICURIO_CONVERTERS=true - - KEY_CONVERTER=io.apicurio.registry.utils.converter.AvroConverter - - VALUE_CONVERTER=io.apicurio.registry.utils.converter.AvroConverter - - CONNECT_KEY_CONVERTER=io.apicurio.registry.utils.converter.AvroConverter - - CONNECT_KEY_CONVERTER_APICURIO.REGISTRY_URL=http://schemaregistry:8080/apis/registry/v2 - - CONNECT_KEY_CONVERTER_APICURIO_REGISTRY_AUTO-REGISTER=true - - CONNECT_KEY_CONVERTER_APICURIO_REGISTRY_FIND-LATEST=true - - CONNECT_VALUE_CONVERTER=io.apicurio.registry.utils.converter.AvroConverter - - CONNECT_VALUE_CONVERTER_APICURIO_REGISTRY_URL=http://schemaregistry:8080/apis/registry/v2 - - CONNECT_VALUE_CONVERTER_APICURIO_REGISTRY_AUTO-REGISTER=true - - CONNECT_VALUE_CONVERTER_APICURIO_REGISTRY_FIND-LATEST=true + - KEY_CONVERTER=io.confluent.connect.avro.AvroConverter + - VALUE_CONVERTER=io.confluent.connect.avro.AvroConverter - KAFKA_OPTS=-javaagent:/kafka/etc/jmx_prometheus_javaagent.jar=8080:/kafka/etc/config.yml - JMXHOST=localhost - JMXPORT=1976 @@ -283,7 +274,7 @@ services: build: context: ../../docker/grafana args: - GRAFANA_VERSION: 7.5.5 + GRAFANA_VERSION: latest #container_name: grafana #image: grafana/grafana restart: "no" diff --git a/doc/setup.md b/doc/setup.md index d4a884940..078666c2c 100644 --- a/doc/setup.md +++ b/doc/setup.md @@ -26,14 +26,23 @@ It will start: 4. RedPanda 5. clickhouse-kafka-sink-connector 6. Clickhouse +7. Confluent Schema registry or Apicurio Schema registry The `start-docker-compose.sh` by default uses the `latest` tag, you could also pass the docker tag to the script. Altinity sink images are tagged on every successful build with the following format(yyyy-mm-dd) Example(2022-07-19) + +### MySQL: ```bash cd deploy/docker ./start-docker-compose.sh ``` +### Postgres: +```bash +cd deploy/docker +docker-compose -f docker-compose-postgresql.yaml up +``` + ### Start Docker-compose with a specific docker tag ```bash cd deploy/docker @@ -48,14 +57,35 @@ cd deploy/docker # Source connector After all the docker containers are up and running, execute the following command to create the Debezium MySQL connector. -Make sure MySQL master/slave is up and running before executing the following script. + +Make sure MySQL master/slave is up and running before executing the following script.\ + +### MySQL: +```bash + ../deploy/debezium-connector-setup-schema-registry.sh +``` [debezium-connector-setup-schema-registry.sh](../deploy/debezium-connector-setup-schema-registry.sh) +### Postgres(Using Apicurio): +```bash +../deploy/debezium-connector-setup-schema-registry.sh postgres apicurio +``` + # Sink Connector After the source connector is created, execute the script [sink-connector-setup-schema-registry.sh](../deploy/sink-connector-setup-schema-registry.sh) to create the Clickhouse Sink connector using Kafka connect REST API +### MySQL: +```bash + ../deploy/sink-connector-setup-schema-registry.sh +``` +### Postgres(Using Apicurio): +```bash +../deploy/sink-connector-setup-schema-registry.sh postgres apicurio +``` + + # Deleting connectors The source connector can be deleted using the following script [debezium-delete.sh](../deploy/debezium-delete.sh) @@ -66,7 +96,7 @@ The sink connector can be deleted using the following script # References Kafka Connect REST API - (https://docs.confluent.io/platform/current/connect/references/restapi.html) -[docker-compose.yaml]: ../deploy/docker/docker-compose.yaml +[docker-compose.yaml]: ../deploy/docker/docker-compose-apicurio-schema-registry.yaml [Dockerfile]: ../docker/Dockerfile-sink-on-debezium-base-image From 64b89f6aa749f9aaa4250b367ad84f501b18355e Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Tue, 9 Aug 2022 17:33:13 -0400 Subject: [PATCH 13/13] Add max.batch.size and max.queue.size to debezium configuration to improve lag. --- deploy/debezium-connector-setup-schema-registry.sh | 4 +++- deploy/debezium-connector-setup-sysbench.sh | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/deploy/debezium-connector-setup-schema-registry.sh b/deploy/debezium-connector-setup-schema-registry.sh index 53779ee2d..347789c37 100755 --- a/deploy/debezium-connector-setup-schema-registry.sh +++ b/deploy/debezium-connector-setup-schema-registry.sh @@ -142,7 +142,9 @@ else "topic.creation.default.replication.factor": 1, "topic.creation.default.partitions": 1, - "provide.transaction.metadata": "true" + "provide.transaction.metadata": "true", + "max.batch.size": 128000, + "max.queue.size": 512000 } } EOF diff --git a/deploy/debezium-connector-setup-sysbench.sh b/deploy/debezium-connector-setup-sysbench.sh index 69560c0c8..c7a21f5ae 100755 --- a/deploy/debezium-connector-setup-sysbench.sh +++ b/deploy/debezium-connector-setup-sysbench.sh @@ -106,7 +106,9 @@ else "topic.creation.default.replication.factor": 1, "topic.creation.default.partitions": 6, - "provide.transaction.metadata": "true" + "provide.transaction.metadata": "true", + "max.batch.size": 128000, + "max.queue.size": 512000 } } EOF